Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions be/src/vec/core/call_on_type_index.h
Original file line number Diff line number Diff line change
Expand Up @@ -276,6 +276,9 @@ bool call_on_index_and_number_data_type(TypeIndex number, F&& f) {
case TypeIndex::Float64:
return f(TypePair<DataTypeNumber<Float64>, T>());

case TypeIndex::TimeV2:
return f(TypePair<DataTypeTimeV2, T>());

case TypeIndex::Decimal32:
return f(TypePair<DataTypeDecimal<Decimal32>, T>());
case TypeIndex::Decimal64:
Expand Down
152 changes: 107 additions & 45 deletions be/src/vec/functions/function_cast.h
Original file line number Diff line number Diff line change
Expand Up @@ -24,16 +24,13 @@
#include <fmt/format.h>
#include <gen_cpp/FrontendService_types.h>
#include <glog/logging.h>
#include <stddef.h>
#include <stdint.h>

#include <algorithm>
#include <atomic>
#include <boost/iterator/iterator_facade.hpp>
#include <cmath>
#include <cstddef>
#include <cstdint>
#include <functional>
#include <iterator>
#include <limits>
#include <memory>
#include <ostream>
Expand Down Expand Up @@ -91,22 +88,16 @@
#include "vec/data_types/data_type_time_v2.h"
#include "vec/data_types/serde/data_type_serde.h"
#include "vec/functions/function.h"
#include "vec/functions/function_convert_tz.h"
#include "vec/functions/function_helpers.h"
#include "vec/io/reader_buffer.h"
#include "vec/runtime/time_value.h"
#include "vec/runtime/vdatetime_value.h"
#include "vec/utils/util.hpp"

class DateLUTImpl;

namespace doris {
namespace vectorized {
namespace doris::vectorized {
template <typename T>
class ColumnDecimal;
} // namespace vectorized
} // namespace doris

namespace doris::vectorized {
/** Type conversion functions.
* toType - conversion in "natural way";
*/
Expand Down Expand Up @@ -423,11 +414,58 @@ struct ConvertImpl {
}
} else {
if constexpr (IsDateTimeV2Type<FromDataType>) {
static_cast_set(
vec_to[i],
reinterpret_cast<const DateV2Value<DateTimeV2ValueType>&>(
vec_from[i])
.to_int64());
if constexpr (std::is_same_v<ToDataType, DataTypeTimeV2>) {
// datetimev2 to timev2
auto dtmv2 = binary_cast<UInt64, DateV2Value<DateTimeV2ValueType>>(
col_from->get_data()[i]);

const auto* type = assert_cast<const DataTypeDateTimeV2*>(
block.get_by_position(arguments[0]).type.get());
auto scale = type->get_scale();
const auto* to_type = assert_cast<const DataTypeTimeV2*>(
block.get_by_position(result).type.get());
UInt32 to_scale = to_type->get_scale();

uint32_t hour = dtmv2.hour();
uint32_t minute = dtmv2.minute();
uint32_t second = dtmv2.second();
uint32_t microseconds = dtmv2.microsecond();
if (to_scale < scale) { // need to round
// e.g. scale reduce to 4, means we need to round the last 2 digits
// 999956: 56 > 100/2, then round up to 1000000
uint32_t divisor = common::exp10_i64(6 - to_scale);
uint32_t remainder = microseconds % divisor;
microseconds = (microseconds / divisor) * divisor;
if (remainder >= divisor / 2) {
// do rounding up
microseconds += divisor;
}
}

// carry on if microseconds >= 1000000
if (microseconds >= 1000000) {
microseconds -= 1000000;
second += 1;
if (second >= 60) {
second -= 60;
minute += 1;
if (minute >= 60) {
minute -= 60;
hour += 1;
}
}
}

auto time = TimeValue::limit_with_bound(
TimeValue::make_time(hour, minute, second, microseconds));
col_to->get_data()[i] = time;
} else {
static_cast_set(
vec_to[i],
reinterpret_cast<const DateV2Value<DateTimeV2ValueType>&>(
vec_from[i])
.to_int64());
}
} else {
static_cast_set(vec_to[i],
reinterpret_cast<const DateV2Value<DateV2ValueType>&>(
Expand Down Expand Up @@ -498,8 +536,8 @@ struct ConvertImplToTimeType {
using FromFieldType = typename FromDataType::FieldType;
using ToFieldType = typename ToDataType::FieldType;

static Status execute(Block& block, const ColumnNumbers& arguments, size_t result,
size_t /*input_rows_count*/) {
static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
uint32_t result, size_t /*input_rows_count*/) {
const ColumnWithTypeAndName& named_from = block.get_by_position(arguments[0]);

using ColVecFrom =
Expand Down Expand Up @@ -527,29 +565,53 @@ struct ConvertImplToTimeType {
col_null_map_to = ColumnUInt8::create(size, 0);
auto& vec_null_map_to = col_null_map_to->get_data();

UInt32 from_precision = 0;
UInt32 from_scale = 0;
UInt32 to_precision = NumberTraits::max_ascii_len<Int64>();
if constexpr (IsDecimalNumber<FromFieldType>) {
const auto& from_decimal_type = assert_cast<const FromDataType&>(*named_from.type);
from_precision = from_decimal_type.get_precision();
from_scale = from_decimal_type.get_scale();
if constexpr (std::is_same_v<FromDataType, DataTypeTimeV2>) {
DateValueType current_date_value;
current_date_value.from_unixtime(context->state()->timestamp_ms() / 1000,
context->state()->timezone_obj());
uint32_t scale = 0;
// Only DateTimeV2 has scale
if (std::is_same_v<ToDataType, DataTypeDateTimeV2>) {
scale = remove_nullable(block.get_by_position(result).type)->get_scale();
}
// According to MySQL rules, when casting time type to date/datetime,
// the current date is added to the time
// So here we need to clear the time part
current_date_value.reset_time_part();
for (size_t i = 0; i < size; ++i) {
auto& date_value = reinterpret_cast<DateValueType&>(vec_to[i]);
date_value = current_date_value;
int64_t microsecond = TimeValue::round_time(vec_from[i], scale);
// Only TimeV2 type needs microseconds
if constexpr (IsTimeV2Type<ToDataType>) {
vec_null_map_to[i] = !date_value.template date_add_interval<MICROSECOND>(
TimeInterval {MICROSECOND, microsecond, false});
} else {
vec_null_map_to[i] =
!date_value.template date_add_interval<SECOND>(TimeInterval {
SECOND, microsecond / TimeValue::ONE_SECOND_MICROSECONDS,
false});
}

// DateType of VecDateTimeValue should cast to date
if constexpr (IsDateType<ToDataType>) {
date_value.cast_to_date();
} else if constexpr (IsDateTimeType<ToDataType>) {
date_value.to_datetime();
}
}
} else {
for (size_t i = 0; i < size; ++i) {
auto& date_value = reinterpret_cast<DateValueType&>(vec_to[i]);
vec_null_map_to[i] = !date_value.from_date_int64(int64_t(vec_from[i]));
// DateType of VecDateTimeValue should cast to date
if constexpr (IsDateType<ToDataType>) {
date_value.cast_to_date();
} else if constexpr (IsDateTimeType<ToDataType>) {
date_value.to_datetime();
}
}
}
bool narrow_integral = to_precision < (from_precision - from_scale);
std::visit(
[&](auto narrow_integral) {
for (size_t i = 0; i < size; ++i) {
auto& date_value = reinterpret_cast<DateValueType&>(vec_to[i]);
vec_null_map_to[i] = !date_value.from_date_int64(int64_t(vec_from[i]));
// DateType of VecDateTimeValue should cast to date
if constexpr (IsDateType<ToDataType>) {
date_value.cast_to_date();
} else if constexpr (IsDateTimeType<ToDataType>) {
date_value.to_datetime();
}
}
},
make_bool_variant(narrow_integral));
block.get_by_position(result).column =
ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
} else {
Expand Down Expand Up @@ -1687,10 +1749,10 @@ class FunctionConvertFromString : public IFunction {
};

template <typename ToDataType, typename Name>
class FunctionConvertToTimeType : public IFunction {
class FunctionConvertFromDatelikeType : public IFunction {
public:
static constexpr auto name = Name::name;
static FunctionPtr create() { return std::make_shared<FunctionConvertToTimeType>(); }
static FunctionPtr create() { return std::make_shared<FunctionConvertFromDatelikeType>(); }

String get_name() const override { return name; }

Expand All @@ -1715,7 +1777,7 @@ class FunctionConvertToTimeType : public IFunction {
using RightDataType = typename Types::RightType;

ret_status = ConvertImplToTimeType<LeftDataType, RightDataType, Name>::execute(
block, arguments, result, input_rows_count);
context, block, arguments, result, input_rows_count);
return true;
};

Expand Down Expand Up @@ -1788,7 +1850,7 @@ class FunctionCast final : public IFunctionBase {
check_and_get_data_type<DataTypeDate>(from_type.get()) ||
check_and_get_data_type<DataTypeDateV2>(from_type.get()) ||
check_and_get_data_type<DataTypeDateTimeV2>(from_type.get()))) {
function = FunctionConvertToTimeType<DataType, NameCast>::create();
function = FunctionConvertFromDatelikeType<DataType, NameCast>::create();
} else {
function = FunctionTo<DataType>::Type::create();
}
Expand Down
46 changes: 46 additions & 0 deletions be/src/vec/runtime/time_value.h
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
#include "runtime/define_primitive_type.h"
#include "runtime/primitive_type.h"
#include "util/date_func.h"
#include "vec/common/int_exp.h"
#include "vec/data_types/data_type_time.h"

namespace doris {
Expand All @@ -34,10 +35,43 @@ class TimeValue {
constexpr static int64_t ONE_HOUR_MICROSECONDS = 60 * ONE_MINUTE_MICROSECONDS;
constexpr static int64_t ONE_MINUTE_SECONDS = 60;
constexpr static int64_t ONE_HOUR_SECONDS = 60 * ONE_MINUTE_SECONDS;
constexpr static uint32_t MICROS_SCALE = 6;
constexpr static int64_t MAX_TIME =
3024000LL * ONE_SECOND_MICROSECONDS - 1; // 840:00:00 - 1ms -> 838:59:59.999999

using TimeType = typename PrimitiveTypeTraits<TYPE_TIMEV2>::CppType;
using ColumnTime = vectorized::DataTypeTimeV2::ColumnType;

static int64_t round_time(TimeType value, uint32_t scale) {
auto time = (int64_t)value;
DCHECK(scale <= MICROS_SCALE);
int64_t factor = common::exp10_i64(6 - scale);
int64_t rounded_value = (time >= 0) ? (time + factor / 2) / factor * factor
: (time - factor / 2) / factor * factor;
return rounded_value;
}

// Construct time based on hour/minute/second/microsecond
template <bool CHECK = false>
static TimeType make_time(int64_t hour, int64_t minute, int64_t second, int64_t microsecond = 0,
bool negative = false) {
if constexpr (CHECK) {
// the max time value is 838:59:59.999999
if (std::abs(hour) > 838 || std::abs(minute) >= 60 || std::abs(second) >= 60 ||
std::abs(microsecond) >= 1000000) [[unlikely]] {
throw Exception(ErrorCode::INVALID_ARGUMENT,
"Invalid time value: hour={}, minute={}, second={}, microsecond={}",
hour, minute, second, microsecond);
}
}
DCHECK(hour >= 0 && minute >= 0 && second >= 0 && microsecond >= 0)
<< "Hour, minute, second and microsecond must be non-negative but got " << hour
<< ":" << minute << ":" << second << "." << microsecond;
int64_t value = (hour * ONE_HOUR_MICROSECONDS) + (minute * ONE_MINUTE_MICROSECONDS) +
(second * ONE_SECOND_MICROSECONDS) + microsecond;
return static_cast<TimeType>(negative ? -value : value);
}

// refer to https://dev.mysql.com/doc/refman/5.7/en/time.html
// the time value between '-838:59:59' and '838:59:59'
/// TODO: Why is the time type stored as double? Can we directly use int64 and remove the time limit?
Expand Down Expand Up @@ -67,6 +101,18 @@ class TimeValue {
static int32_t second(TimeType time) {
return (check_over_max_time(time) / ONE_SECOND_MICROSECONDS) % ONE_MINUTE_SECONDS;
}

// refer to https://dev.mysql.com/doc/refman/5.7/en/time.html
// the time value between '-838:59:59' and '838:59:59'
static TimeType limit_with_bound(TimeType time) {
if (time > MAX_TIME) {
return MAX_TIME;
}
if (time < -MAX_TIME) {
return -MAX_TIME;
}
return time;
}
};

} // namespace doris
15 changes: 15 additions & 0 deletions be/src/vec/runtime/vdatetime_value.h
Original file line number Diff line number Diff line change
Expand Up @@ -442,6 +442,12 @@ class VecDateTimeValue { // Now this type is a temp solution with little changes
return _hour * SECOND_PER_HOUR + _minute * SECOND_PER_MINUTE + _second;
}

void reset_time_part() {
_hour = 0;
_minute = 0;
_second = 0;
}

bool check_loss_accuracy_cast_to_date() {
auto loss_accuracy = _hour != 0 || _minute != 0 || _second != 0;
cast_to_date();
Expand Down Expand Up @@ -934,6 +940,15 @@ class DateV2Value {
return hour() * SECOND_PER_HOUR + minute() * SECOND_PER_MINUTE + second();
}

void reset_time_part() {
if constexpr (is_datetime) {
date_v2_value_.hour_ = 0;
date_v2_value_.minute_ = 0;
date_v2_value_.second_ = 0;
date_v2_value_.microsecond_ = 0;
}
}

int64_t time_part_to_microsecond() const {
return time_part_to_seconds() * 1000 * 1000 + microsecond();
}
Expand Down
50 changes: 50 additions & 0 deletions be/test/runtime/time_value_test.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#include <cctz/civil_time.h>
#include <cctz/time_zone.h>
#include <gtest/gtest.h>
#include <vec/runtime/time_value.h>

namespace doris {

TEST(TimeValueTest, make_time) {
int64_t hour = 1;
int64_t minute = 2;
int64_t second = 3;
TimeValue::TimeType time = TimeValue::make_time(hour, minute, second);
EXPECT_EQ(time, 3723000000);
}

TEST(TimeValueTest, round_time) {
// 01:02:03.500000 -> 01:02:04.000000
EXPECT_EQ(TimeValue::round_time(TimeValue::make_time(1, 2, 3, 500000), 0),
TimeValue::make_time(1, 2, 4));

// 01:02:03.499999 -> 01:01:03.000000
EXPECT_EQ(TimeValue::round_time(TimeValue::make_time(1, 2, 3, 499999), 0),
TimeValue::make_time(1, 2, 3));

// -01:02:03.500000 -> -01:01:04.000000
EXPECT_EQ(TimeValue::round_time(-TimeValue::make_time(1, 2, 3, 500000), 0),
-TimeValue::make_time(1, 2, 4));

// -01:02:03.499999 -> -01:01:03.000000
EXPECT_EQ(TimeValue::round_time(-TimeValue::make_time(1, 2, 3, 499999), 0),
-TimeValue::make_time(1, 2, 3));
}
} // namespace doris
Loading
Loading