Merge pull request #10722 from Enmk/DateTime64_fixes

function toStartOfSecond(DateTime64) -> DateTime64
This commit is contained in:
alexey-milovidov 2020-05-11 20:20:58 +03:00 committed by GitHub
commit 1e70230991
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 169 additions and 62 deletions

View File

@ -50,9 +50,10 @@ struct DecimalComponents
* If `scale` is to big (scale > maxPrecision<DecimalType::NativeType>), result is undefined. * If `scale` is to big (scale > maxPrecision<DecimalType::NativeType>), result is undefined.
*/ */
template <typename DecimalType> template <typename DecimalType>
DecimalType decimalFromComponentsWithMultiplier(const typename DecimalType::NativeType & whole, inline DecimalType decimalFromComponentsWithMultiplier(
const typename DecimalType::NativeType & fractional, const typename DecimalType::NativeType & whole,
typename DecimalType::NativeType scale_multiplier) const typename DecimalType::NativeType & fractional,
typename DecimalType::NativeType scale_multiplier)
{ {
using T = typename DecimalType::NativeType; using T = typename DecimalType::NativeType;
const auto fractional_sign = whole < 0 ? -1 : 1; const auto fractional_sign = whole < 0 ? -1 : 1;
@ -70,8 +71,10 @@ DecimalType decimalFromComponentsWithMultiplier(const typename DecimalType::Nati
* @see `decimalFromComponentsWithMultiplier` for details. * @see `decimalFromComponentsWithMultiplier` for details.
*/ */
template <typename DecimalType> template <typename DecimalType>
DecimalType decimalFromComponents( inline DecimalType decimalFromComponents(
const typename DecimalType::NativeType & whole, const typename DecimalType::NativeType & fractional, UInt32 scale) const typename DecimalType::NativeType & whole,
const typename DecimalType::NativeType & fractional,
UInt32 scale)
{ {
using T = typename DecimalType::NativeType; using T = typename DecimalType::NativeType;
@ -82,8 +85,9 @@ DecimalType decimalFromComponents(
* @see `decimalFromComponentsWithMultiplier` for details. * @see `decimalFromComponentsWithMultiplier` for details.
*/ */
template <typename DecimalType> template <typename DecimalType>
DecimalType decimalFromComponents( inline DecimalType decimalFromComponents(
const DecimalComponents<typename DecimalType::NativeType> & components, UInt32 scale) const DecimalComponents<typename DecimalType::NativeType> & components,
UInt32 scale)
{ {
return decimalFromComponents<DecimalType>(components.whole, components.fractional, scale); return decimalFromComponents<DecimalType>(components.whole, components.fractional, scale);
} }
@ -92,8 +96,9 @@ DecimalType decimalFromComponents(
* This is an optimization to reduce number of calls to scaleMultiplier on known scale. * This is an optimization to reduce number of calls to scaleMultiplier on known scale.
*/ */
template <typename DecimalType> template <typename DecimalType>
DecimalComponents<typename DecimalType::NativeType> splitWithScaleMultiplier( inline DecimalComponents<typename DecimalType::NativeType> splitWithScaleMultiplier(
const DecimalType & decimal, typename DecimalType::NativeType scale_multiplier) const DecimalType & decimal,
typename DecimalType::NativeType scale_multiplier)
{ {
using T = typename DecimalType::NativeType; using T = typename DecimalType::NativeType;
const auto whole = decimal.value / scale_multiplier; const auto whole = decimal.value / scale_multiplier;
@ -106,7 +111,7 @@ DecimalComponents<typename DecimalType::NativeType> splitWithScaleMultiplier(
/// Split decimal into components: whole and fractional part, @see `DecimalComponents` for details. /// Split decimal into components: whole and fractional part, @see `DecimalComponents` for details.
template <typename DecimalType> template <typename DecimalType>
DecimalComponents<typename DecimalType::NativeType> split(const DecimalType & decimal, UInt32 scale) inline DecimalComponents<typename DecimalType::NativeType> split(const DecimalType & decimal, UInt32 scale)
{ {
if (scale == 0) if (scale == 0)
{ {
@ -121,7 +126,7 @@ DecimalComponents<typename DecimalType::NativeType> split(const DecimalType & de
* If scale is to big, result is undefined. * If scale is to big, result is undefined.
*/ */
template <typename DecimalType> template <typename DecimalType>
typename DecimalType::NativeType getWholePart(const DecimalType & decimal, size_t scale) inline typename DecimalType::NativeType getWholePart(const DecimalType & decimal, size_t scale)
{ {
if (scale == 0) if (scale == 0)
return decimal.value; return decimal.value;
@ -129,24 +134,34 @@ typename DecimalType::NativeType getWholePart(const DecimalType & decimal, size_
return decimal.value / scaleMultiplier<typename DecimalType::NativeType>(scale); return decimal.value / scaleMultiplier<typename DecimalType::NativeType>(scale);
} }
template <typename DecimalType, bool keep_sign = false>
inline typename DecimalType::NativeType getFractionalPartWithScaleMultiplier(
const DecimalType & decimal,
typename DecimalType::NativeType scale_multiplier)
{
using T = typename DecimalType::NativeType;
T result = decimal.value;
if constexpr (!keep_sign)
if (result < T(0))
result = -result;
return result % scale_multiplier;
}
/** Get fractional part from decimal /** Get fractional part from decimal
* *
* Result is always positive. * Result is always positive.
* If scale is to big, result is undefined. * If scale is to big, result is undefined.
*/ */
template <typename DecimalType> template <typename DecimalType>
typename DecimalType::NativeType getFractionalPart(const DecimalType & decimal, size_t scale) inline typename DecimalType::NativeType getFractionalPart(const DecimalType & decimal, size_t scale)
{ {
using T = typename DecimalType::NativeType;
if (scale == 0) if (scale == 0)
return 0; return 0;
T result = decimal.value; return getFractionalPartWithScaleMultiplier(decimal, scaleMultiplier<typename DecimalType::NativeType>(scale));
if (result < T(0))
result *= T(-1);
return result % scaleMultiplier<T>(scale);
} }
} }

View File

@ -45,40 +45,74 @@ public:
bool equals(const IDataType & rhs) const override; bool equals(const IDataType & rhs) const override;
}; };
/** Tansform-type wrapper for DateTime64, applies given Transform to DateTime64 value or only to a whole part of it.
/** Basic wrapper for Tansform-types for DateTime64.
* *
* Allows reusing existing Transform (that takes DateTime-values as UInt32) with DateTime64-values, * Depending on what overloads of Transform::execute() are available, when called with DateTime64 value,
* by discarding fractional part and producing SAME return type as original Transform. * invokes Transform::execute() with:
* * whole part of DateTime64 value, discarding fractional part.
* * DateTime64 value and scale factor.
* *
* Such Transfotm-types are commonly used in Date/DateTime manipulation functions, * Suitable Transfotm-types are commonly used in Date/DateTime manipulation functions,
* and implement static execute fucntion with following signature: * and should implement static (or const) fucntion with following signatures:
* R execute(UInt32, T, const DateLUTImpl &) * R execute(UInt32 whole_value, ... , const TimeZoneImpl &)
* OR
* R execute(DateTime64 value, Int64 scale_factor, ... , const TimeZoneImpl &)
* *
* Wehere R and T could be arbitrary types. * Wehere R and T could be arbitrary types.
*/ */
template <typename Transform> template <typename Transform>
class DateTime64BasicTransformWrapper : public Transform class TransformDateTime64 : public Transform
{ {
private:
// ExtractName::name is Transform::name or default.
struct ExtractName
{
template<typename U> static constexpr auto getName(...) { return ""; }
template<typename U, typename = std::void_t<decltype(U::name)>> static constexpr auto getName(void*) { return U::name; }
static constexpr auto name = getName<Transform>(nullptr);
};
// Detect if Transform::execute is const or static method
// with signature defined by template args (ignoring result type).
template<typename = void, typename... Args>
struct TransformHasExecuteOverload : std::false_type {};
template<typename... Args>
struct TransformHasExecuteOverload<std::void_t<decltype(std::declval<Transform>().execute(std::declval<Args>()...))>, Args...>
: std::true_type {};
template<typename... Args>
static constexpr bool TransformHasExecuteOverload_v = TransformHasExecuteOverload<void, Args...>::value;
public: public:
static constexpr auto name = ExtractName::name;
using Transform::execute; using Transform::execute;
explicit DateTime64BasicTransformWrapper(UInt32 scale_) // non-explicit constructor to allow creating from scale value (or with no scale at all), indispensable in some contexts.
TransformDateTime64(UInt32 scale_ = 0)
: scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale_)) : scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale_))
{} {}
template <typename T> template <typename ... Args>
auto execute(DateTime64 t, T v, const DateLUTImpl & time_zone) const inline auto execute(const DateTime64 & t, Args && ... args) const
{ {
const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); const auto transform = static_cast<const Transform *>(this);
return static_cast<const Transform *>(this)->execute(
static_cast<UInt32>(components.whole), v, time_zone); if constexpr (TransformHasExecuteOverload_v<DateTime64, decltype(scale_multiplier), Args...>)
{
return transform->execute(t, scale_multiplier, std::forward<Args>(args)...);
}
else
{
const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
return transform->execute(static_cast<UInt32>(components.whole), std::forward<Args>(args)...);
}
} }
private: private:
UInt32 scale_multiplier = 1; DateTime64::NativeType scale_multiplier = 1;
}; };
} }

View File

@ -175,6 +175,41 @@ struct ToStartOfMinuteImpl
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
}; };
// Rounding towards negative infinity.
// 1.01 => 1.00
// -1.01 => -2
struct ToStartOfSecondImpl
{
static constexpr auto name = "toStartOfSecond";
static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &)
{
auto fractional_with_sign = DecimalUtils::getFractionalPartWithScaleMultiplier<DateTime64, true>(datetime64, scale_multiplier);
// given that scale is 3, scale_multiplier is 1000
// for DateTime64 value of 123.456:
// 123456 - 456 = 123000
// for DateTime64 value of -123.456:
// -123456 - (1000 + (-456)) = -124000
if (fractional_with_sign < 0)
fractional_with_sign += scale_multiplier;
return datetime64 - fractional_with_sign;
}
static inline UInt32 execute(UInt32, const DateLUTImpl &)
{
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
static inline UInt32 execute(UInt16, const DateLUTImpl &)
{
return dateIsNotSupported(name);
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfFiveMinuteImpl struct ToStartOfFiveMinuteImpl
{ {
static constexpr auto name = "toStartOfFiveMinute"; static constexpr auto name = "toStartOfFiveMinute";

View File

@ -110,7 +110,7 @@ public:
{ {
CustomWeekTransformImpl<DataTypeDateTime64, ToDataType>::execute( CustomWeekTransformImpl<DataTypeDateTime64, ToDataType>::execute(
block, arguments, result, input_rows_count, block, arguments, result, input_rows_count,
DateTime64BasicTransformWrapper<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()}); TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
} }
else else
throw Exception( throw Exception(

View File

@ -16,26 +16,6 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
} }
template<class Transform>
struct WithDateTime64Converter : public Transform
{
UInt8 scale;
Transform transform;
explicit WithDateTime64Converter(UInt8 scale_, Transform transform_ = {})
: scale(scale_),
transform(std::move(transform_))
{}
inline auto execute(DataTypeDateTime64::FieldType t, const DateLUTImpl & time_zone) const
{
auto x = DateTime64(t);
auto res = transform.execute(static_cast<UInt32>(DecimalUtils::getWholePart(x, scale)), time_zone);
return res;
}
};
/// See DateTimeTransforms.h /// See DateTimeTransforms.h
template <typename ToDataType, typename Transform> template <typename ToDataType, typename Transform>
class FunctionDateOrDateTimeToSomething : public IFunction class FunctionDateOrDateTimeToSomething : public IFunction
@ -90,7 +70,13 @@ public:
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>) if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
return std::make_shared<ToDataType>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); return std::make_shared<ToDataType>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>) if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
return std::make_shared<ToDataType>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); {
Int64 scale = DataTypeDateTime64::default_scale;
if (const auto * dt64 = checkAndGetDataType<DataTypeDateTime64>(arguments[0].type.get()))
scale = dt64->getScale();
return std::make_shared<ToDataType>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
}
else else
return std::make_shared<ToDataType>(); return std::make_shared<ToDataType>();
} }
@ -110,16 +96,14 @@ public:
else if (which.isDateTime64()) else if (which.isDateTime64())
{ {
const auto scale = static_cast<const DataTypeDateTime64 *>(from_type)->getScale(); const auto scale = static_cast<const DataTypeDateTime64 *>(from_type)->getScale();
WithDateTime64Converter<Transform> transformer(scale); const TransformDateTime64<Transform> transformer(scale);
DateTimeTransformImpl<DataTypeDateTime64, ToDataType, decltype(transformer)>::execute(block, arguments, result, input_rows_count, transformer);
DateTimeTransformImpl<DataTypeDateTime64, ToDataType, WithDateTime64Converter<Transform>>::execute(block, arguments, result, input_rows_count, transformer);
} }
else else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
} }
bool hasInformationAboutMonotonicity() const override bool hasInformationAboutMonotonicity() const override
{ {
return true; return true;

View File

@ -11,6 +11,7 @@ void registerFunctionToDayOfWeek(FunctionFactory &);
void registerFunctionToDayOfYear(FunctionFactory &); void registerFunctionToDayOfYear(FunctionFactory &);
void registerFunctionToHour(FunctionFactory &); void registerFunctionToHour(FunctionFactory &);
void registerFunctionToMinute(FunctionFactory &); void registerFunctionToMinute(FunctionFactory &);
void registerFunctionToStartOfSecond(FunctionFactory &);
void registerFunctionToSecond(FunctionFactory &); void registerFunctionToSecond(FunctionFactory &);
void registerFunctionToStartOfDay(FunctionFactory &); void registerFunctionToStartOfDay(FunctionFactory &);
void registerFunctionToMonday(FunctionFactory &); void registerFunctionToMonday(FunctionFactory &);
@ -84,6 +85,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionToStartOfMonth(factory); registerFunctionToStartOfMonth(factory);
registerFunctionToStartOfQuarter(factory); registerFunctionToStartOfQuarter(factory);
registerFunctionToStartOfYear(factory); registerFunctionToStartOfYear(factory);
registerFunctionToStartOfSecond(factory);
registerFunctionToStartOfMinute(factory); registerFunctionToStartOfMinute(factory);
registerFunctionToStartOfFiveMinute(factory); registerFunctionToStartOfFiveMinute(factory);
registerFunctionToStartOfTenMinutes(factory); registerFunctionToStartOfTenMinutes(factory);

View File

@ -311,7 +311,7 @@ private:
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>) if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
{ {
const auto transform = DateTime64BasicTransformWrapper<Transform<unit>>{from_datatype.getScale()}; const auto transform = TransformDateTime64<Transform<unit>>{from_datatype.getScale()};
for (size_t i = 0; i != size; ++i) for (size_t i = 0; i != size; ++i)
result_data[i] = transform.execute(time_data[i], num_units, time_zone); result_data[i] = transform.execute(time_data[i], num_units, time_zone);
} }

View File

@ -0,0 +1,18 @@
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionFactory.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
using FunctionToStartOfSecond = FunctionDateOrDateTimeToSomething<DataTypeDateTime64, ToStartOfSecondImpl>;
void registerFunctionToStartOfSecond(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStartOfSecond>();
}
}

View File

@ -0,0 +1,6 @@
2019-09-16 16:20:11.000 DateTime64(3, \'UTC\')
2019-09-16 19:20:11 DateTime64(0, \'UTC\')
2019-09-16 19:20:11.000 DateTime64(3, \'UTC\')
2019-09-16 19:20:11.000000000 DateTime64(9, \'UTC\')
non-const column
2019-09-16 19:20:11.000 DateTime64(3, \'UTC\')

View File

@ -0,0 +1,13 @@
-- Error cases
SELECT toStartOfSecond('123'); -- {serverError 43}
SELECT toStartOfSecond(now()); -- {serverError 43}
SELECT toStartOfSecond(); -- {serverError 42}
SELECT toStartOfSecond(now64(), 123); -- {serverError 43}
WITH toDateTime64('2019-09-16 19:20:11', 3) AS dt64 SELECT toStartOfSecond(dt64, 'UTC') AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11', 0, 'UTC') AS dt64 SELECT toStartOfSecond(dt64) AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11.123', 3, 'UTC') AS dt64 SELECT toStartOfSecond(dt64) AS res, toTypeName(res);
WITH toDateTime64('2019-09-16 19:20:11.123', 9, 'UTC') AS dt64 SELECT toStartOfSecond(dt64) AS res, toTypeName(res);
SELECT 'non-const column';
WITH toDateTime64('2019-09-16 19:20:11.123', 3, 'UTC') AS dt64 SELECT toStartOfSecond(materialize(dt64)) AS res, toTypeName(res);