mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #10722 from Enmk/DateTime64_fixes
function toStartOfSecond(DateTime64) -> DateTime64
This commit is contained in:
commit
1e70230991
@ -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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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";
|
||||||
|
@ -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(
|
||||||
|
@ -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;
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
18
src/Functions/toStartOfSecond.cpp
Normal file
18
src/Functions/toStartOfSecond.cpp
Normal 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>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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\')
|
13
tests/queries/0_stateless/01269_toStartOfSecond.sql
Normal file
13
tests/queries/0_stateless/01269_toStartOfSecond.sql
Normal 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);
|
Loading…
Reference in New Issue
Block a user