未验证 提交 7d77a7ba 编写于 作者: N Nikolai Kochetov 提交者: GitHub

Merge pull request #16952 from Enmk/DateTime64_fixes_comparison

Fixed comparison of DateTime64 with different scales
......@@ -4,6 +4,7 @@
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include "registerAggregateFunctions.h"
......
......@@ -100,6 +100,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, c
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
//if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
}
......
......@@ -237,6 +237,8 @@ template <typename T, typename TResult, typename Data, AggregateFunctionSumType
class AggregateFunctionSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionSum<T, TResult, Data, Type>>
{
public:
static constexpr bool DateTime64Supported = false;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<TResult>, DataTypeNumber<TResult>>;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<TResult>, ColumnVector<TResult>>;
......
......@@ -135,6 +135,8 @@ static IAggregateFunction * createWithDecimalType(const IDataType & argument_typ
if (which.idx == TypeIndex::Decimal64) return new AggregateFunctionTemplate<Decimal64>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal128) return new AggregateFunctionTemplate<Decimal128>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal256) return new AggregateFunctionTemplate<Decimal256>(std::forward<TArgs>(args)...);
if constexpr (AggregateFunctionTemplate<DateTime64>::DateTime64Supported)
if (which.idx == TypeIndex::DateTime64) return new AggregateFunctionTemplate<DateTime64>(std::forward<TArgs>(args)...);
return nullptr;
}
......@@ -146,6 +148,8 @@ static IAggregateFunction * createWithDecimalType(const IDataType & argument_typ
if (which.idx == TypeIndex::Decimal64) return new AggregateFunctionTemplate<Decimal64, Data>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal128) return new AggregateFunctionTemplate<Decimal128, Data>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal256) return new AggregateFunctionTemplate<Decimal256, Data>(std::forward<TArgs>(args)...);
if constexpr (AggregateFunctionTemplate<DateTime64, Data>::DateTime64Supported)
if (which.idx == TypeIndex::DateTime64) return new AggregateFunctionTemplate<DateTime64, Data>(std::forward<TArgs>(args)...);
return nullptr;
}
......
......@@ -310,6 +310,9 @@ protected:
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data*>(place); }
public:
// Derived class can `override` this to flag that DateTime64 is not supported.
static constexpr bool DateTime64Supported = true;
IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_)
: IAggregateFunctionHelper<Derived>(argument_types_, parameters_) {}
......
......@@ -370,4 +370,5 @@ template class ColumnDecimal<Decimal32>;
template class ColumnDecimal<Decimal64>;
template class ColumnDecimal<Decimal128>;
template class ColumnDecimal<Decimal256>;
template class ColumnDecimal<DateTime64>;
}
......@@ -114,15 +114,15 @@ private:
static std::enable_if_t<IsDecimalNumber<T> && IsDecimalNumber<U>, Shift>
getScales(const DataTypePtr & left_type, const DataTypePtr & right_type)
{
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
const DataTypeDecimalBase<T> * decimal0 = checkDecimalBase<T>(*left_type);
const DataTypeDecimalBase<U> * decimal1 = checkDecimalBase<U>(*right_type);
Shift shift;
if (decimal0 && decimal1)
{
auto result_type = decimalResultType<false, false>(*decimal0, *decimal1);
shift.a = static_cast<CompareInt>(result_type.scaleFactorFor(*decimal0, false).value);
shift.b = static_cast<CompareInt>(result_type.scaleFactorFor(*decimal1, false).value);
auto result_type = DecimalUtils::binaryOpResult<false, false>(*decimal0, *decimal1);
shift.a = static_cast<CompareInt>(result_type.scaleFactorFor(decimal0->getTrait(), false).value);
shift.b = static_cast<CompareInt>(result_type.scaleFactorFor(decimal1->getTrait(), false).value);
}
else if (decimal0)
shift.b = static_cast<CompareInt>(decimal0->getScaleMultiplier().value);
......@@ -137,7 +137,7 @@ private:
getScales(const DataTypePtr & left_type, const DataTypePtr &)
{
Shift shift;
const DataTypeDecimal<T> * decimal0 = checkDecimal<T>(*left_type);
const DataTypeDecimalBase<T> * decimal0 = checkDecimalBase<T>(*left_type);
if (decimal0)
shift.b = static_cast<CompareInt>(decimal0->getScaleMultiplier().value);
return shift;
......@@ -148,7 +148,7 @@ private:
getScales(const DataTypePtr &, const DataTypePtr & right_type)
{
Shift shift;
const DataTypeDecimal<U> * decimal1 = checkDecimal<U>(*right_type);
const DataTypeDecimalBase<U> * decimal1 = checkDecimalBase<U>(*right_type);
if (decimal1)
shift.a = static_cast<CompareInt>(decimal1->getScaleMultiplier().value);
return shift;
......
......@@ -11,9 +11,13 @@
namespace DB
{
template <typename T>
class DataTypeNumber;
namespace ErrorCodes
{
extern const int DECIMAL_OVERFLOW;
extern const int ARGUMENT_OUT_OF_BOUND;
}
namespace DecimalUtils
......@@ -23,6 +27,7 @@ static constexpr size_t minPrecision() { return 1; }
template <typename T> static constexpr size_t maxPrecision() { return 0; }
template <> constexpr size_t maxPrecision<Decimal32>() { return 9; }
template <> constexpr size_t maxPrecision<Decimal64>() { return 18; }
template <> constexpr size_t maxPrecision<DateTime64>() { return 18; }
template <> constexpr size_t maxPrecision<Decimal128>() { return 38; }
template <> constexpr size_t maxPrecision<Decimal256>() { return 76; }
......@@ -31,7 +36,7 @@ inline auto scaleMultiplier(UInt32 scale)
{
if constexpr (std::is_same_v<T, Int32> || std::is_same_v<T, Decimal32>)
return common::exp10_i32(scale);
else if constexpr (std::is_same_v<T, Int64> || std::is_same_v<T, Decimal64>)
else if constexpr (std::is_same_v<T, Int64> || std::is_same_v<T, Decimal64> || std::is_same_v<T, DateTime64>)
return common::exp10_i64(scale);
else if constexpr (std::is_same_v<T, Int128> || std::is_same_v<T, Decimal128>)
return common::exp10_i128(scale);
......@@ -51,6 +56,30 @@ struct DecimalComponents
T fractional;
};
/// Traits used for determining final Type/Precision/Scale for certain math operations on decimals.
template <typename T>
struct DataTypeDecimalTrait
{
using FieldType = T;
const UInt32 precision;
const UInt32 scale;
DataTypeDecimalTrait(UInt32 precision_, UInt32 scale_)
: precision(precision_),
scale(scale_)
{}
/// @returns multiplier for U to become T with correct scale
template <typename U>
T scaleFactorFor(const DataTypeDecimalTrait<U> & x, bool) const
{
if (scale < x.scale)
throw Exception("Decimal result's scale is less than argument's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
const UInt32 scale_delta = scale - x.scale; /// scale_delta >= 0
return DecimalUtils::scaleMultiplier<typename T::NativeType>(scale_delta);
}
};
/** Make a decimal value from whole and fractional components with given scale multiplier.
* where scale_multiplier = scaleMultiplier<T>(scale)
* this is to reduce number of calls to scaleMultiplier when scale is known.
......@@ -211,6 +240,35 @@ To convertTo(const DecimalType & decimal, size_t scale)
}
}
template <bool is_multiply, bool is_division, typename T, typename U, template <typename> typename DecimalType>
inline auto binaryOpResult(const DecimalType<T> & tx, const DecimalType<U> & ty)
{
UInt32 scale{};
if constexpr (is_multiply)
scale = tx.getScale() + ty.getScale();
else if constexpr (is_division)
scale = tx.getScale();
else
scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale());
if constexpr (sizeof(T) < sizeof(U))
return DataTypeDecimalTrait<U>(DecimalUtils::maxPrecision<U>(), scale);
else
return DataTypeDecimalTrait<T>(DecimalUtils::maxPrecision<T>(), scale);
}
template <bool, bool, typename T, typename U, template <typename> typename DecimalType>
inline const DataTypeDecimalTrait<T> binaryOpResult(const DecimalType<T> & tx, const DataTypeNumber<U> &)
{
return DataTypeDecimalTrait<T>(DecimalUtils::maxPrecision<T>(), tx.getScale());
}
template <bool, bool, typename T, typename U, template <typename> typename DecimalType>
inline const DataTypeDecimalTrait<U> binaryOpResult(const DataTypeNumber<T> &, const DecimalType<U> & ty)
{
return DataTypeDecimalTrait<U>(DecimalUtils::maxPrecision<U>(), ty.getScale());
}
}
}
......@@ -592,6 +592,10 @@ template <> bool decimalEqual(Decimal256 x, Decimal256 y, UInt32 x_scale, UInt32
template <> bool decimalLess(Decimal256 x, Decimal256 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(Decimal256 x, Decimal256 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
template <> bool decimalEqual(DateTime64 x, DateTime64 y, UInt32 x_scale, UInt32 y_scale) { return decEqual(x, y, x_scale, y_scale); }
template <> bool decimalLess(DateTime64 x, DateTime64 y, UInt32 x_scale, UInt32 y_scale) { return decLess(x, y, x_scale, y_scale); }
template <> bool decimalLessOrEqual(DateTime64 x, DateTime64 y, UInt32 x_scale, UInt32 y_scale) { return decLessOrEqual(x, y, x_scale, y_scale); }
inline void writeText(const Null &, WriteBuffer & buf)
{
writeText(std::string("Null"), buf);
......
......@@ -193,10 +193,12 @@ template <> struct NearestFieldTypeImpl<Decimal32> { using Type = DecimalField<D
template <> struct NearestFieldTypeImpl<Decimal64> { using Type = DecimalField<Decimal64>; };
template <> struct NearestFieldTypeImpl<Decimal128> { using Type = DecimalField<Decimal128>; };
template <> struct NearestFieldTypeImpl<Decimal256> { using Type = DecimalField<Decimal256>; };
template <> struct NearestFieldTypeImpl<DateTime64> { using Type = DecimalField<DateTime64>; };
template <> struct NearestFieldTypeImpl<DecimalField<Decimal32>> { using Type = DecimalField<Decimal32>; };
template <> struct NearestFieldTypeImpl<DecimalField<Decimal64>> { using Type = DecimalField<Decimal64>; };
template <> struct NearestFieldTypeImpl<DecimalField<Decimal128>> { using Type = DecimalField<Decimal128>; };
template <> struct NearestFieldTypeImpl<DecimalField<Decimal256>> { using Type = DecimalField<Decimal256>; };
template <> struct NearestFieldTypeImpl<DecimalField<DateTime64>> { using Type = DecimalField<DateTime64>; };
template <> struct NearestFieldTypeImpl<Float32> { using Type = Float64; };
template <> struct NearestFieldTypeImpl<Float64> { using Type = Float64; };
template <> struct NearestFieldTypeImpl<const char *> { using Type = String; };
......@@ -731,6 +733,7 @@ template <> struct Field::TypeToEnum<DecimalField<Decimal32>>{ static const Type
template <> struct Field::TypeToEnum<DecimalField<Decimal64>>{ static const Types::Which value = Types::Decimal64; };
template <> struct Field::TypeToEnum<DecimalField<Decimal128>>{ static const Types::Which value = Types::Decimal128; };
template <> struct Field::TypeToEnum<DecimalField<Decimal256>>{ static const Types::Which value = Types::Decimal256; };
template <> struct Field::TypeToEnum<DecimalField<DateTime64>>{ static const Types::Which value = Types::Decimal64; };
template <> struct Field::TypeToEnum<AggregateFunctionStateData>{ static const Types::Which value = Types::AggregateFunctionState; };
template <> struct Field::TypeToEnum<UInt256> { static const Types::Which value = Types::UInt256; };
template <> struct Field::TypeToEnum<Int256> { static const Types::Which value = Types::Int256; };
......
......@@ -184,29 +184,44 @@ using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
using Decimal256 = Decimal<Int256>;
using DateTime64 = Decimal64;
// Distinguishable type to allow function resultion/deduction based on value type,
// but also relatively easy to convert to/from Decimal64.
class DateTime64 : public Decimal64
{
public:
using Base = Decimal64;
using Base::Base;
DateTime64(const Base & v)
: Base(v)
{}
};
template <> struct TypeName<Decimal32> { static constexpr const char * get() { return "Decimal32"; } };
template <> struct TypeName<Decimal64> { static constexpr const char * get() { return "Decimal64"; } };
template <> struct TypeName<Decimal128> { static constexpr const char * get() { return "Decimal128"; } };
template <> struct TypeName<Decimal256> { static constexpr const char * get() { return "Decimal256"; } };
template <> struct TypeName<DateTime64> { static constexpr const char * get() { return "DateTime64"; } };
template <> struct TypeId<Decimal32> { static constexpr const TypeIndex value = TypeIndex::Decimal32; };
template <> struct TypeId<Decimal64> { static constexpr const TypeIndex value = TypeIndex::Decimal64; };
template <> struct TypeId<Decimal128> { static constexpr const TypeIndex value = TypeIndex::Decimal128; };
template <> struct TypeId<Decimal256> { static constexpr const TypeIndex value = TypeIndex::Decimal256; };
template <> struct TypeId<DateTime64> { static constexpr const TypeIndex value = TypeIndex::DateTime64; };
template <typename T> constexpr bool IsDecimalNumber = false;
template <> inline constexpr bool IsDecimalNumber<Decimal32> = true;
template <> inline constexpr bool IsDecimalNumber<Decimal64> = true;
template <> inline constexpr bool IsDecimalNumber<Decimal128> = true;
template <> inline constexpr bool IsDecimalNumber<Decimal256> = true;
template <> inline constexpr bool IsDecimalNumber<DateTime64> = true;
template <typename T> struct NativeType { using Type = T; };
template <> struct NativeType<Decimal32> { using Type = Int32; };
template <> struct NativeType<Decimal64> { using Type = Int64; };
template <> struct NativeType<Decimal128> { using Type = Int128; };
template <> struct NativeType<Decimal256> { using Type = Int256; };
template <> struct NativeType<DateTime64> { using Type = Int64; };
template <typename T> constexpr bool OverBigInt = false;
template <> inline constexpr bool OverBigInt<Int256> = true;
......@@ -275,6 +290,15 @@ namespace std
}
};
template <>
struct hash<DB::DateTime64>
{
size_t operator()(const DB::DateTime64 & x) const
{
return std::hash<std::decay_t<decltype(x)>::NativeType>()(x);
}
};
template <>
struct hash<DB::Decimal256>
......
......@@ -141,7 +141,7 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f)
{
case TypeIndex::Date: return callOnBasicType<UInt16, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::DateTime: return callOnBasicType<UInt32, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::DateTime64: return callOnBasicType<Decimal64, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::DateTime64: return callOnBasicType<DateTime64, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
default:
break;
}
......
......@@ -16,6 +16,7 @@ namespace DB
class DataTypeDateTime64 final : public DataTypeDecimalBase<DateTime64>, public TimezoneMixin
{
public:
using Base = DataTypeDecimalBase<DateTime64>;
static constexpr UInt8 default_scale = 3;
static constexpr auto family_name = "DateTime64";
......
......@@ -102,5 +102,6 @@ template class DataTypeDecimalBase<Decimal32>;
template class DataTypeDecimalBase<Decimal64>;
template class DataTypeDecimalBase<Decimal128>;
template class DataTypeDecimalBase<Decimal256>;
template class DataTypeDecimalBase<DateTime64>;
}
......@@ -150,39 +150,45 @@ public:
static T getScaleMultiplier(UInt32 scale);
inline DecimalUtils::DataTypeDecimalTrait<T> getTrait() const
{
return {precision, scale};
}
protected:
const UInt32 precision;
const UInt32 scale;
};
template <typename T>
inline const DataTypeDecimalBase<T> * checkDecimalBase(const IDataType & data_type)
{
if (isColumnedAsDecimalT<T>(data_type))
return static_cast<const DataTypeDecimalBase<T> *>(&data_type);
return nullptr;
}
template <bool is_multiply, bool is_division, typename T, typename U, template <typename> typename DecimalType>
inline auto decimalResultType(const DecimalType<T> & tx, const DecimalType<U> & ty)
{
UInt32 scale{};
if constexpr (is_multiply)
scale = tx.getScale() + ty.getScale();
else if constexpr (is_division)
scale = tx.getScale();
else
scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale());
if constexpr (sizeof(T) < sizeof(U))
return DecimalType<U>(DecimalUtils::maxPrecision<U>(), scale);
else
return DecimalType<T>(DecimalUtils::maxPrecision<T>(), scale);
const auto result_trait = DecimalUtils::binaryOpResult<is_multiply, is_division>(tx, ty);
return DecimalType<typename decltype(result_trait)::FieldType>(result_trait.precision, result_trait.scale);
}
template <bool, bool, typename T, typename U, template <typename> typename DecimalType>
inline const DecimalType<T> decimalResultType(const DecimalType<T> & tx, const DataTypeNumber<U> &)
template <bool is_multiply, bool is_division, typename T, typename U, template <typename> typename DecimalType>
inline const DecimalType<T> decimalResultType(const DecimalType<T> & tx, const DataTypeNumber<U> & ty)
{
return DecimalType<T>(DecimalUtils::maxPrecision<T>(), tx.getScale());
const auto result_trait = DecimalUtils::binaryOpResult<is_multiply, is_division>(tx, ty);
return DecimalType<typename decltype(result_trait)::FieldType>(result_trait.precision, result_trait.scale);
}
template <bool, bool, typename T, typename U, template <typename> typename DecimalType>
inline const DecimalType<U> decimalResultType(const DataTypeNumber<T> &, const DecimalType<U> & ty)
template <bool is_multiply, bool is_division, typename T, typename U, template <typename> typename DecimalType>
inline const DecimalType<U> decimalResultType(const DataTypeNumber<T> & tx, const DecimalType<U> & ty)
{
return DecimalType<U>(DecimalUtils::maxPrecision<U>(), ty.getScale());
const auto result_trait = DecimalUtils::binaryOpResult<is_multiply, is_division>(tx, ty);
return DecimalType<typename decltype(result_trait)::FieldType>(result_trait.precision, result_trait.scale);
}
template <template <typename> typename DecimalType>
......
......@@ -606,6 +606,14 @@ inline bool isColumnedAsDecimal(const T & data_type)
return which.isDecimal() || which.isDateTime64();
}
// Same as isColumnedAsDecimal but also checks value type of underlyig column.
template <typename T, typename DataType>
inline bool isColumnedAsDecimalT(const DataType & data_type)
{
const WhichDataType which(data_type);
return (which.isDecimal() || which.isDateTime64()) && which.idx == TypeId<T>::value;
}
template <typename T>
inline bool isString(const T & data_type)
{
......
......@@ -486,6 +486,8 @@ static void writeFieldsToColumn(
write_data_to_column(casted_decimal_128_column, Decimal128(), Decimal128());
else if (ColumnDecimal<Decimal256> * casted_decimal_256_column = typeid_cast<ColumnDecimal<Decimal256> *>(&column_to))
write_data_to_column(casted_decimal_256_column, Decimal256(), Decimal256());
else if (ColumnDecimal<DateTime64> * casted_datetime_64_column = typeid_cast<ColumnDecimal<DateTime64> *>(&column_to))
write_data_to_column(casted_datetime_64_column, DateTime64(), DateTime64());
else if (ColumnInt32 * casted_int32_column = typeid_cast<ColumnInt32 *>(&column_to))
{
for (size_t index = 0; index < rows_data.size(); ++index)
......
......@@ -370,7 +370,14 @@ end:
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW, typename T>
void readIntText(T & x, ReadBuffer & buf)
{
readIntTextImpl<T, void, check_overflow>(x, buf);
if constexpr (IsDecimalNumber<T>)
{
readIntText<check_overflow>(x.value, buf);
}
else
{
readIntTextImpl<T, void, check_overflow>(x, buf);
}
}
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::CHECK_OVERFLOW, typename T>
......@@ -379,11 +386,6 @@ bool tryReadIntText(T & x, ReadBuffer & buf)
return readIntTextImpl<T, bool, check_overflow>(x, buf);
}
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW, typename T>
void readIntText(Decimal<T> & x, ReadBuffer & buf)
{
readIntText<check_overflow>(x.value, buf);
}
/** More efficient variant (about 1.5 times on real dataset).
* Differs in following:
......
......@@ -39,6 +39,7 @@ void callWithType(TypeIndex which, F && f)
case TypeIndex::Decimal32: return f(Decimal32());
case TypeIndex::Decimal64: return f(Decimal64());
case TypeIndex::Decimal128: return f(Decimal128());
case TypeIndex::DateTime64: return f(DateTime64());
default:
break;
}
......@@ -153,6 +154,9 @@ std::optional<TypeIndex> AsofRowRefs::getTypeSize(const IColumn & asof_column, s
case TypeIndex::Decimal128:
size = sizeof(Decimal128);
return idx;
case TypeIndex::DateTime64:
size = sizeof(DateTime64);
return idx;
default:
break;
}
......
......@@ -228,7 +228,8 @@ public:
Entry<Float64>::LookupPtr,
Entry<Decimal32>::LookupPtr,
Entry<Decimal64>::LookupPtr,
Entry<Decimal128>::LookupPtr>;
Entry<Decimal128>::LookupPtr,
Entry<DateTime64>::LookupPtr>;
AsofRowRefs() {}
AsofRowRefs(TypeIndex t);
......
......@@ -319,7 +319,7 @@ ColumnPtr fillColumnWithRandomData(
case TypeIndex::DateTime64:
{
auto column = type->createColumn();
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal64> &>(*column);
auto & column_concrete = typeid_cast<ColumnDecimal<DateTime64> &>(*column);
column_concrete.getData().resize(limit);
UInt64 range = (1ULL << 32) * intExp10(typeid_cast<const DataTypeDateTime64 &>(*type).getScale());
......
2015-05-18 07:40:29.123456 2019-09-16 19:20:33.123 0 0 0 1 1 1
2019-09-16 19:20:33.123000 2019-09-16 19:20:33.123 0 1 1 1 0 0
2019-09-16 19:20:33.123456 2015-05-18 07:40:29.123 1 1 0 0 0 1
CREATE TABLE datetime64_cmp
(
dt6 DateTime64(6, 'UTC'),
dt3 DateTime64(3, 'UTC')
) ENGINE = Memory;
INSERT INTO datetime64_cmp
VALUES ('2019-09-16 19:20:33.123000', '2019-09-16 19:20:33.123'), ('2019-09-16 19:20:33.123456', '2015-05-18 07:40:29.123'), ('2015-05-18 07:40:29.123456', '2019-09-16 19:20:33.123');
-- Compare equal and unequal values of different precicion/scale
SELECT
dt6, dt3,
dt6 > dt3,
dt6 >= dt3,
dt6 = dt3,
dt6 <= dt3,
dt6 < dt3,
dt6 != dt3
FROM datetime64_cmp
ORDER BY
dt6, dt3;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册