Functions {to,from}UnixTimestamp64{Milli,Micro,Nano}

* Family of functions to convert DateTime64 to Int64 and back, usefull for
   imporintg/exporting raw timestamp values of certain precision.
 * Docs
 * Tests
 * Fixed ya.make
This commit is contained in:
Vasily Nemkov 2020-05-08 17:59:58 +03:00
parent e9d348ebf9
commit 28a2cc0a82
25 changed files with 614 additions and 29 deletions

View File

@ -568,4 +568,90 @@ Result:
└───────────────────────┘
```
## toUnixTimestamp64Milli
## toUnixTimestamp64Micro
## toUnixTimestamp64Nano
Converts a `DateTime64` to a `Int64` value with fixed sub-second precision. Input value is scaled up or down appropriately depending on it precision. Please note that output value is a timestamp in UTC, not in timezone of `DateTime64`.
**Syntax**
``` sql
toUnixTimestamp64Milli(value)
```
**Parameters**
- `value` — DateTime64 value with any precision.
**Returned value**
- `value` converted to the `Int64` data type.
**Examples**
Query:
``` sql
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64
SELECT toUnixTimestamp64Milli(dt64)
```
Result:
``` text
┌─toUnixTimestamp64Milli(dt64)─┐
│ 1568650812345 │
└──────────────────────────────┘
```
``` sql
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64
SELECT toUnixTimestamp64Nano(dt64)
```
Result:
``` text
┌─toUnixTimestamp64Nano(dt64)─┐
│ 1568650812345678000 │
└─────────────────────────────┘
```
## fromUnixTimestamp64Milli
## fromUnixTimestamp64Micro
## fromUnixTimestamp64Nano
Converts an `Int64` to a `DateTime64` value with fixed sub-second precision and optional timezone. Input value is scaled up or down appropriately depending on it's precision. Please note that input value is treated as UTC timestamp, not timestamp at given (or implicit) timezone.
**Syntax**
``` sql
fromUnixTimestamp64Milli(value [, ti])
```
**Parameters**
- `value``Int64` value with any precision.
- `timezone``String` (optional) timezone name of the result.
**Returned value**
- `value` converted to the `DateTime64` data type.
**Examples**
``` sql
WITH CAST(1234567891011, 'Int64') AS i64
SELECT fromUnixTimestamp64Milli(i64, 'UTC')
```
``` text
┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐
│ 2009-02-13 23:31:31.011 │
└──────────────────────────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/query_language/functions/type_conversion_functions/) <!--hide-->

View File

@ -153,6 +153,8 @@ public:
const T & getElement(size_t n) const { return data[n]; }
T & getElement(size_t n) { return data[n]; }
UInt32 getScale() const {return scale;}
protected:
Container data;
UInt32 scale;

View File

@ -224,13 +224,19 @@ public:
}
template <typename ... TAllocatorParams>
void push_back_raw(const char * ptr, TAllocatorParams &&... allocator_params)
void push_back_raw(const void * ptr, TAllocatorParams &&... allocator_params)
{
push_back_raw_many(1, ptr, std::forward<TAllocatorParams>(allocator_params)...);
}
template <typename ... TAllocatorParams>
void push_back_raw_many(size_t number_of_items, const void * ptr, TAllocatorParams &&... allocator_params)
{
if (unlikely(c_end == c_end_of_storage))
reserveForNextSize(std::forward<TAllocatorParams>(allocator_params)...);
reserve(number_of_items, std::forward<TAllocatorParams>(allocator_params)...);
memcpy(c_end, ptr, ELEMENT_SIZE);
c_end += byte_size(1);
memcpy(c_end, ptr, ELEMENT_SIZE * number_of_items);
c_end += byte_size(number_of_items);
}
void protect()

View File

@ -72,7 +72,7 @@ template <typename T> bool inline operator<= (T a, const UInt128 b) { return UIn
template <typename T> bool inline operator< (T a, const UInt128 b) { return UInt128(a) < b; }
template <> inline constexpr bool IsNumber<UInt128> = true;
template <> struct TypeName<UInt128> { static const char * get() { return "UInt128"; } };
template <> struct TypeName<UInt128> { static constexpr const char * get() { return "UInt128"; } };
template <> struct TypeId<UInt128> { static constexpr const TypeIndex value = TypeIndex::UInt128; };
struct UInt128Hash

View File

@ -85,17 +85,17 @@ template <> inline constexpr bool IsNumber<Float64> = true;
template <typename T> struct TypeName;
template <> struct TypeName<UInt8> { static const char * get() { return "UInt8"; } };
template <> struct TypeName<UInt16> { static const char * get() { return "UInt16"; } };
template <> struct TypeName<UInt32> { static const char * get() { return "UInt32"; } };
template <> struct TypeName<UInt64> { static const char * get() { return "UInt64"; } };
template <> struct TypeName<Int8> { static const char * get() { return "Int8"; } };
template <> struct TypeName<Int16> { static const char * get() { return "Int16"; } };
template <> struct TypeName<Int32> { static const char * get() { return "Int32"; } };
template <> struct TypeName<Int64> { static const char * get() { return "Int64"; } };
template <> struct TypeName<Float32> { static const char * get() { return "Float32"; } };
template <> struct TypeName<Float64> { static const char * get() { return "Float64"; } };
template <> struct TypeName<String> { static const char * get() { return "String"; } };
template <> struct TypeName<UInt8> { static constexpr const char * get() { return "UInt8"; } };
template <> struct TypeName<UInt16> { static constexpr const char * get() { return "UInt16"; } };
template <> struct TypeName<UInt32> { static constexpr const char * get() { return "UInt32"; } };
template <> struct TypeName<UInt64> { static constexpr const char * get() { return "UInt64"; } };
template <> struct TypeName<Int8> { static constexpr const char * get() { return "Int8"; } };
template <> struct TypeName<Int16> { static constexpr const char * get() { return "Int16"; } };
template <> struct TypeName<Int32> { static constexpr const char * get() { return "Int32"; } };
template <> struct TypeName<Int64> { static constexpr const char * get() { return "Int64"; } };
template <> struct TypeName<Float32> { static constexpr const char * get() { return "Float32"; } };
template <> struct TypeName<Float64> { static constexpr const char * get() { return "Float64"; } };
template <> struct TypeName<String> { static constexpr const char * get() { return "String"; } };
template <typename T> struct TypeId;
template <> struct TypeId<UInt8> { static constexpr const TypeIndex value = TypeIndex::UInt8; };
@ -115,7 +115,7 @@ using Strings = std::vector<String>;
using Int128 = __int128;
template <> inline constexpr bool IsNumber<Int128> = true;
template <> struct TypeName<Int128> { static const char * get() { return "Int128"; } };
template <> struct TypeName<Int128> { static constexpr const char * get() { return "Int128"; } };
template <> struct TypeId<Int128> { static constexpr const TypeIndex value = TypeIndex::Int128; };
/// Own FieldType for Decimal.
@ -161,9 +161,9 @@ using Decimal128 = Decimal<Int128>;
using DateTime64 = Decimal64;
template <> struct TypeName<Decimal32> { static const char * get() { return "Decimal32"; } };
template <> struct TypeName<Decimal64> { static const char * get() { return "Decimal64"; } };
template <> struct TypeName<Decimal128> { static const char * get() { return "Decimal128"; } };
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 TypeId<Decimal32> { static constexpr const TypeIndex value = TypeIndex::Decimal32; };
template <> struct TypeId<Decimal64> { static constexpr const TypeIndex value = TypeIndex::Decimal64; };
@ -183,7 +183,7 @@ template <> inline Int32 Decimal32::getScaleMultiplier(UInt32 scale) { return co
template <> inline Int64 Decimal64::getScaleMultiplier(UInt32 scale) { return common::exp10_i64(scale); }
template <> inline Int128 Decimal128::getScaleMultiplier(UInt32 scale) { return common::exp10_i128(scale); }
inline const char * getTypeName(TypeIndex idx)
inline constexpr const char * getTypeName(TypeIndex idx)
{
switch (idx)
{

View File

@ -17,7 +17,9 @@ class DataTypeDateTime64 final : public DataTypeDecimalBase<DateTime64>, public
{
public:
static constexpr UInt8 default_scale = 3;
static constexpr auto family_name = "DateTime64";
static constexpr auto type_id = TypeIndex::DateTime64;
explicit DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name = "");
@ -26,7 +28,7 @@ public:
const char * getFamilyName() const override { return family_name; }
std::string doGetName() const override;
TypeIndex getTypeId() const override { return TypeIndex::DateTime64; }
TypeIndex getTypeId() const override { return type_id; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;

View File

@ -19,12 +19,15 @@ class DataTypeNumberBase : public DataTypeWithSimpleSerialization
public:
static constexpr bool is_parametric = false;
using FieldType = T;
static constexpr auto type_id = TypeId<T>::value;
static constexpr auto family_name = TypeName<T>::get();
using ColumnType = ColumnVector<T>;
const char * getFamilyName() const override { return TypeName<T>::get(); }
TypeIndex getTypeId() const override { return TypeId<T>::value; }
const char * getFamilyName() const override { return family_name; }
TypeIndex getTypeId() const override { return type_id; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;

View File

@ -13,13 +13,14 @@ class DataTypeString final : public IDataType
public:
using FieldType = String;
static constexpr bool is_parametric = false;
static constexpr auto type_id = TypeIndex::String;
const char * getFamilyName() const override
{
return "String";
}
TypeIndex getTypeId() const override { return TypeIndex::String; }
TypeIndex getTypeId() const override { return type_id; }
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;

View File

@ -529,10 +529,15 @@ struct WhichDataType
/// IDataType helpers (alternative for IDataType virtual methods with single point of truth)
inline bool isDate(const DataTypePtr & data_type) { return WhichDataType(data_type).isDate(); }
inline bool isDateOrDateTime(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateOrDateTime(); }
inline bool isDateTime(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateTime(); }
inline bool isDateTime64(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateTime64(); }
template <typename T>
inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); }
template <typename T>
inline bool isDateOrDateTime(const T & data_type) { return WhichDataType(data_type).isDateOrDateTime(); }
template <typename T>
inline bool isDateTime(const T & data_type) { return WhichDataType(data_type).isDateTime(); }
template <typename T>
inline bool isDateTime64(const T & data_type) { return WhichDataType(data_type).isDateTime64(); }
inline bool isEnum(const DataTypePtr & data_type) { return WhichDataType(data_type).isEnum(); }
inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data_type).isDecimal(); }
inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); }
@ -636,6 +641,19 @@ inline bool isCompilableType(const DataTypePtr & data_type)
return data_type->isValueRepresentedByNumber() && !isDecimal(data_type);
}
template <TypeIndex TYPE_IDX, typename DataType>
inline bool isDataType(const DataType & data_type)
{
WhichDataType which(data_type);
return which.idx == TYPE_IDX;
}
template <typename ExpectedDataType, typename DataType>
inline bool isDataType(const DataType & data_type)
{
return isDataType<ExpectedDataType::type_id>(data_type);
}
template <typename DataType> constexpr bool IsDataTypeDecimal = false;
template <typename DataType> constexpr bool IsDataTypeNumber = false;
template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;

View File

@ -0,0 +1,148 @@
#pragma once
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <common/arithmeticOverflow.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int DECIMAL_OVERFLOW;
}
/** Casts DateTim64 to or from Int64 representation narrowed down (or scaled up) to any scale value defined in Impl.
*/
template <typename Impl>
class FunctionUnixTimestamp64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static constexpr auto target_scale = Impl::target_scale;
using SourceDataType = typename Impl::SourceDataType;
using ResultDataType = typename Impl::ResultDataType;
static constexpr bool is_result_datetime64 = std::is_same_v<ResultDataType, DataTypeDateTime64>;
static_assert(std::is_same_v<SourceDataType, DataTypeDateTime64> || std::is_same_v<ResultDataType, DataTypeDateTime64>);
static auto create(const Context &)
{
return std::make_shared<FunctionUnixTimestamp64<Impl>>();
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return is_result_datetime64 ? 2 : 1; }
bool isVariadic() const override { return is_result_datetime64; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if constexpr (is_result_datetime64)
{
validateFunctionArgumentTypes(*this, arguments,
FunctionArgumentDescriptors{{"value", isDataType<SourceDataType>, nullptr, std::string(SourceDataType::family_name).c_str()}},
// optional
FunctionArgumentDescriptors{
// {"precision", isDataType<DataTypeUInt8>, isColumnConst, ("Precision of the result, default is " + std::to_string(target_scale)).c_str()},
{"timezone", isStringOrFixedString, isColumnConst, "Timezone of the result"},
});
const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0);
return std::make_shared<DataTypeDateTime64>(target_scale, timezone);
}
else
{
validateFunctionArgumentTypes(*this, arguments,
FunctionArgumentDescriptors{{"value", isDataType<SourceDataType>, nullptr, std::string(SourceDataType::family_name).c_str()}});
return std::make_shared<DataTypeInt64>();
}
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
using SourceColumnType = typename SourceDataType::ColumnType;
using ResultColumnType = typename ResultDataType::ColumnType;
const auto & src = block.getByPosition(arguments[0]);
auto & res = block.getByPosition(result);
const auto & col = *src.column;
const SourceColumnType * source_col_typed = checkAndGetColumn<SourceColumnType>(col);
if (!source_col_typed && !(source_col_typed = checkAndGetColumnConstData<SourceColumnType>(&col)))
throw Exception("Invalid column type" + col.getName() + " expected "
+ std::string(SourceDataType::family_name),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
res.column = res.type->createColumn();
if (input_rows_count == 0)
return;
auto & result_data = assert_cast<ResultColumnType &>(res.column->assumeMutableRef()).getData();
result_data.reserve(source_col_typed->size());
const auto & source_data = source_col_typed->getData();
const auto scale_diff = getScaleDiff(*checkAndGetDataType<SourceDataType>(src.type.get()), *checkAndGetDataType<ResultDataType>(res.type.get()));
if (scale_diff == 0)
{
static_assert(sizeof(typename SourceColumnType::Container::value_type) == sizeof(typename ResultColumnType::Container::value_type));
// no conversion necessary
result_data.push_back_raw_many(source_data.size(), source_data.data());
}
else if (scale_diff < 0)
{
const Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(std::abs(scale_diff));
for (const auto & v : source_data)
{
Int64 result_value = toDestValue(v);
if (common::mulOverflow(result_value, scale_multiplier, result_value))
throw Exception("Decimal overflow in " + getName(), ErrorCodes::DECIMAL_OVERFLOW);
result_data.push_back(result_value);
}
}
else
{
const Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(scale_diff);
for (const auto & v : source_data)
result_data.push_back(static_cast<Int64>(toDestValue(v) / scale_multiplier));
}
}
private:
static Int64 getScaleDiff(const SourceDataType & src, const ResultDataType & dst)
{
Int64 src_scale = target_scale;
if constexpr (std::is_same_v<SourceDataType, DataTypeDateTime64>)
{
src_scale = src.getScale();
}
Int64 dst_scale = target_scale;
if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime64>)
{
dst_scale = dst.getScale();
}
return src_scale - dst_scale;
}
static auto toDestValue(const DateTime64 & v)
{
return Int64{v.value};
}
template <typename T>
static auto toDestValue(const T & v)
{
return Int64{v};
}
};
}

View File

@ -0,0 +1,20 @@
#include <Functions/FunctionUnixTimestamp64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TransformFromMicro
{
static constexpr auto name = "fromUnixTimestamp64Micro";
static constexpr auto target_scale = 6;
using SourceDataType = DataTypeInt64;
using ResultDataType = DataTypeDateTime64;
};
void registerFromUnixTimestamp64Micro(FunctionFactory & factory)
{
factory.registerFunction<FunctionUnixTimestamp64<TransformFromMicro>>();
}
}

View File

@ -0,0 +1,20 @@
#include <Functions/FunctionUnixTimestamp64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TransformFromMilli
{
static constexpr auto name = "fromUnixTimestamp64Milli";
static constexpr auto target_scale = 3;
using SourceDataType = DataTypeInt64;
using ResultDataType = DataTypeDateTime64;
};
void registerFromUnixTimestamp64Milli(FunctionFactory & factory)
{
factory.registerFunction<FunctionUnixTimestamp64<TransformFromMilli>>();
}
}

View File

@ -0,0 +1,20 @@
#include <Functions/FunctionUnixTimestamp64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TransformFromNano
{
static constexpr auto name = "fromUnixTimestamp64Nano";
static constexpr auto target_scale = 9;
using SourceDataType = DataTypeInt64;
using ResultDataType = DataTypeDateTime64;
};
void registerFromUnixTimestamp64Nano(FunctionFactory & factory)
{
factory.registerFunction<FunctionUnixTimestamp64<TransformFromNano>>();
}
}

View File

@ -37,6 +37,7 @@ void registerFunctionsIntrospection(FunctionFactory &);
void registerFunctionsNull(FunctionFactory &);
void registerFunctionsJSON(FunctionFactory &);
void registerFunctionsConsistentHashing(FunctionFactory & factory);
void registerFunctionsUnixTimestamp64(FunctionFactory & factory);
void registerFunctions()
@ -78,6 +79,7 @@ void registerFunctions()
registerFunctionsJSON(factory);
registerFunctionsIntrospection(factory);
registerFunctionsConsistentHashing(factory);
registerFunctionsUnixTimestamp64(factory);
}
}

View File

@ -0,0 +1,25 @@
namespace DB
{
class FunctionFactory;
void registerToUnixTimestamp64Milli(FunctionFactory &);
void registerToUnixTimestamp64Micro(FunctionFactory &);
void registerToUnixTimestamp64Nano(FunctionFactory &);
void registerFromUnixTimestamp64Milli(FunctionFactory &);
void registerFromUnixTimestamp64Micro(FunctionFactory &);
void registerFromUnixTimestamp64Nano(FunctionFactory &);
void registerFunctionsUnixTimestamp64(FunctionFactory & factory)
{
registerToUnixTimestamp64Milli(factory);
registerToUnixTimestamp64Micro(factory);
registerToUnixTimestamp64Nano(factory);
registerFromUnixTimestamp64Milli(factory);
registerFromUnixTimestamp64Micro(factory);
registerFromUnixTimestamp64Nano(factory);
}
}

View File

@ -0,0 +1,20 @@
#include <Functions/FunctionUnixTimestamp64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TransformToMicro
{
static constexpr auto name = "toUnixTimestamp64Micro";
static constexpr auto target_scale = 6;
using SourceDataType = DataTypeDateTime64;
using ResultDataType = DataTypeInt64;
};
void registerToUnixTimestamp64Micro(FunctionFactory & factory)
{
factory.registerFunction<FunctionUnixTimestamp64<TransformToMicro>>();
}
}

View File

@ -0,0 +1,20 @@
#include <Functions/FunctionUnixTimestamp64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TransformToMilli
{
static constexpr auto name = "toUnixTimestamp64Milli";
static constexpr auto target_scale = 3;
using SourceDataType = DataTypeDateTime64;
using ResultDataType = DataTypeInt64;
};
void registerToUnixTimestamp64Milli(FunctionFactory & factory)
{
factory.registerFunction<FunctionUnixTimestamp64<TransformToMilli>>();
}
}

View File

@ -0,0 +1,20 @@
#include <Functions/FunctionUnixTimestamp64.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
struct TransformToNano
{
static constexpr auto name = "toUnixTimestamp64Nano";
static constexpr auto target_scale = 9;
using SourceDataType = DataTypeDateTime64;
using ResultDataType = DataTypeInt64;
};
void registerToUnixTimestamp64Nano(FunctionFactory & factory)
{
factory.registerFunction<FunctionUnixTimestamp64<TransformToNano>>();
}
}

View File

@ -151,6 +151,9 @@ SRCS(
finalizeAggregation.cpp
formatDateTime.cpp
formatString.cpp
fromUnixTimestamp64Micro.cpp
fromUnixTimestamp64Milli.cpp
fromUnixTimestamp64Nano.cpp
FunctionFactory.cpp
FunctionFQDN.cpp
FunctionHelpers.cpp
@ -168,6 +171,7 @@ SRCS(
FunctionsRound.cpp
FunctionsStringArray.cpp
FunctionsStringSimilarity.cpp
FunctionUnixTimestamp64.h
GatherUtils/concat.cpp
GatherUtils/createArraySink.cpp
GatherUtils/createArraySource.cpp
@ -305,6 +309,7 @@ SRCS(
registerFunctionsStringSearch.cpp
registerFunctionsTuple.cpp
registerFunctionsVisitParam.cpp
registerFunctionsUnixTimestamp64.cpp
reinterpretAsFixedString.cpp
reinterpretAsString.cpp
reinterpretStringAs.cpp
@ -386,6 +391,9 @@ SRCS(
toTimeZone.cpp
toTypeName.cpp
toValidUTF8.cpp
toUnixTimestamp64Micro.cpp
toUnixTimestamp64Milli.cpp
toUnixTimestamp64Nano.cpp
toYear.cpp
toYYYYMM.cpp
toYYYYMMDD.cpp

View File

@ -0,0 +1,5 @@
const column
UTC 1234567891011 2009-02-13 23:31:31.011 1970-01-15 06:56:07.891011 1970-01-01 00:20:34.567891011 DateTime64(9, \'UTC\')
Asia/Makassar 1234567891011 2009-02-14 07:31:31.011 1970-01-15 14:56:07.891011 1970-01-01 08:20:34.567891011 DateTime64(9, \'Asia/Makassar\')
non-const column
1234567891011 2009-02-13 23:31:31.011 1970-01-15 06:56:07.891011 1970-01-01 00:20:34.567891011

View File

@ -0,0 +1,45 @@
-- -- Error cases
SELECT fromUnixTimestamp64Milli(); -- {serverError 42}
SELECT fromUnixTimestamp64Micro(); -- {serverError 42}
SELECT fromUnixTimestamp64Nano(); -- {serverError 42}
SELECT fromUnixTimestamp64Milli('abc'); -- {serverError 43}
SELECT fromUnixTimestamp64Micro('abc'); -- {serverError 43}
SELECT fromUnixTimestamp64Nano('abc'); -- {serverError 43}
SELECT fromUnixTimestamp64Milli('abc', 123); -- {serverError 43}
SELECT fromUnixTimestamp64Micro('abc', 123); -- {serverError 43}
SELECT fromUnixTimestamp64Nano('abc', 123); -- {serverError 43}
SELECT 'const column';
WITH
CAST(1234567891011 AS Int64) AS i64,
'UTC' AS tz
SELECT
tz,
i64,
fromUnixTimestamp64Milli(i64, tz),
fromUnixTimestamp64Micro(i64, tz),
fromUnixTimestamp64Nano(i64, tz) as dt64,
toTypeName(dt64);
WITH
CAST(1234567891011 AS Int64) AS i64,
'Asia/Makassar' AS tz
SELECT
tz,
i64,
fromUnixTimestamp64Milli(i64, tz),
fromUnixTimestamp64Micro(i64, tz),
fromUnixTimestamp64Nano(i64, tz) as dt64,
toTypeName(dt64);
SELECT 'non-const column';
WITH
CAST(1234567891011 AS Int64) AS i64,
'UTC' AS tz
SELECT
i64,
fromUnixTimestamp64Milli(i64, tz),
fromUnixTimestamp64Micro(i64, tz),
fromUnixTimestamp64Nano(i64, tz) as dt64;

View File

@ -0,0 +1,8 @@
const column
2019-09-16 19:20:12.345 1568650812345 1568650812345000 1568650812345000000
2019-09-16 19:20:12.345678 1568650812345 1568650812345678 1568650812345678000
2019-09-16 19:20:12.345678910 1568650812345 1568650812345678 1568650812345678910
non-const column
2019-09-16 19:20:12.345 1568650812345 1568650812345000 1568650812345000000
2019-09-16 19:20:12.345678 1568650812345 1568650812345678 1568650812345678000
2019-09-16 19:20:12.345678910 1568650812345 1568650812345678 1568650812345678910

View File

@ -0,0 +1,33 @@
-- Error cases
SELECT toUnixTimestamp64Milli(); -- {serverError 42}
SELECT toUnixTimestamp64Micro(); -- {serverError 42}
SELECT toUnixTimestamp64Nano(); -- {serverError 42}
SELECT toUnixTimestamp64Milli('abc'); -- {serverError 43}
SELECT toUnixTimestamp64Micro('abc'); -- {serverError 43}
SELECT toUnixTimestamp64Nano('abc'); -- {serverError 43}
SELECT toUnixTimestamp64Milli('abc', 123); -- {serverError 42}
SELECT toUnixTimestamp64Micro('abc', 123); -- {serverError 42}
SELECT toUnixTimestamp64Nano('abc', 123); -- {serverError 42}
SELECT 'const column';
WITH toDateTime64('2019-09-16 19:20:12.345678910', 3) AS dt64
SELECT dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64
SELECT dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 9) AS dt64
SELECT dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
SELECT 'non-const column';
WITH toDateTime64('2019-09-16 19:20:12.345678910', 3) AS x
SELECT materialize(x) as dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS x
SELECT materialize(x) as dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);
WITH toDateTime64('2019-09-16 19:20:12.345678910', 9) AS x
SELECT materialize(x) as dt64, toUnixTimestamp64Milli(dt64), toUnixTimestamp64Micro(dt64), toUnixTimestamp64Nano(dt64);

View File

@ -0,0 +1,9 @@
2019-09-16 19:20:12.345 2019-09-16 19:20:12.345 2019-09-16 19:20:12.345000 2019-09-16 19:20:12.345000000
2019-09-16 19:20:12.345678 2019-09-16 19:20:12.345 2019-09-16 19:20:12.345678 2019-09-16 19:20:12.345678000
2019-09-16 19:20:12.345678910 2019-09-16 19:20:12.345 2019-09-16 19:20:12.345678 2019-09-16 19:20:12.345678910
with explicit timezone
2019-09-16 19:20:12.345 2019-09-16 19:20:12.345 2019-09-16 19:20:12.345000 2019-09-16 19:20:12.345000000 DateTime64(9, \'UTC\')
2019-09-16 19:20:12.345 2019-09-16 19:20:12.345 2019-09-16 19:20:12.345000 2019-09-16 19:20:12.345000000 DateTime64(9, \'Asia/Makassar\')
1234567891011 1234567891011 1234567891011 1234567891011
with explicit timezone
1234567891011 1234567891011 1234567891011 1234567891011 Int64

View File

@ -0,0 +1,64 @@
WITH
toDateTime64('2019-09-16 19:20:12.345678910', 3) AS dt64
SELECT
dt64,
fromUnixTimestamp64Milli(toUnixTimestamp64Milli(dt64)),
fromUnixTimestamp64Micro(toUnixTimestamp64Micro(dt64)),
fromUnixTimestamp64Nano(toUnixTimestamp64Nano(dt64));
WITH
toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64
SELECT
dt64,
fromUnixTimestamp64Milli(toUnixTimestamp64Milli(dt64)),
fromUnixTimestamp64Micro(toUnixTimestamp64Micro(dt64)),
fromUnixTimestamp64Nano(toUnixTimestamp64Nano(dt64));
WITH
toDateTime64('2019-09-16 19:20:12.345678910', 9) AS dt64
SELECT
dt64,
fromUnixTimestamp64Milli(toUnixTimestamp64Milli(dt64)),
fromUnixTimestamp64Micro(toUnixTimestamp64Micro(dt64)),
fromUnixTimestamp64Nano(toUnixTimestamp64Nano(dt64));
SELECT 'with explicit timezone';
WITH
'UTC' as timezone,
toDateTime64('2019-09-16 19:20:12.345678910', 3, timezone) AS dt64
SELECT
dt64,
fromUnixTimestamp64Milli(toUnixTimestamp64Milli(dt64), timezone),
fromUnixTimestamp64Micro(toUnixTimestamp64Micro(dt64), timezone),
fromUnixTimestamp64Nano(toUnixTimestamp64Nano(dt64), timezone) AS v,
toTypeName(v);
WITH
'Asia/Makassar' as timezone,
toDateTime64('2019-09-16 19:20:12.345678910', 3, timezone) AS dt64
SELECT
dt64,
fromUnixTimestamp64Milli(toUnixTimestamp64Milli(dt64), timezone),
fromUnixTimestamp64Micro(toUnixTimestamp64Micro(dt64), timezone),
fromUnixTimestamp64Nano(toUnixTimestamp64Nano(dt64), timezone) AS v,
toTypeName(v);
WITH
CAST(1234567891011 AS Int64) AS val
SELECT
val,
toUnixTimestamp64Milli(fromUnixTimestamp64Milli(val)),
toUnixTimestamp64Micro(fromUnixTimestamp64Micro(val)),
toUnixTimestamp64Nano(fromUnixTimestamp64Nano(val));
SELECT 'with explicit timezone';
WITH
'UTC' as timezone,
CAST(1234567891011 AS Int64) AS val
SELECT
val,
toUnixTimestamp64Milli(fromUnixTimestamp64Milli(val, timezone)),
toUnixTimestamp64Micro(fromUnixTimestamp64Micro(val, timezone)),
toUnixTimestamp64Nano(fromUnixTimestamp64Nano(val, timezone)) AS v,
toTypeName(v);