mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
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:
parent
e9d348ebf9
commit
28a2cc0a82
@ -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-->
|
||||
|
@ -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;
|
||||
|
@ -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()
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
148
src/Functions/FunctionUnixTimestamp64.h
Normal file
148
src/Functions/FunctionUnixTimestamp64.h
Normal 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};
|
||||
}
|
||||
};
|
||||
|
||||
}
|
20
src/Functions/fromUnixTimestamp64Micro.cpp
Normal file
20
src/Functions/fromUnixTimestamp64Micro.cpp
Normal 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>>();
|
||||
}
|
||||
|
||||
}
|
20
src/Functions/fromUnixTimestamp64Milli.cpp
Normal file
20
src/Functions/fromUnixTimestamp64Milli.cpp
Normal 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>>();
|
||||
}
|
||||
|
||||
}
|
20
src/Functions/fromUnixTimestamp64Nano.cpp
Normal file
20
src/Functions/fromUnixTimestamp64Nano.cpp
Normal 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>>();
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
25
src/Functions/registerFunctionsUnixTimestamp64.cpp
Normal file
25
src/Functions/registerFunctionsUnixTimestamp64.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
20
src/Functions/toUnixTimestamp64Micro.cpp
Normal file
20
src/Functions/toUnixTimestamp64Micro.cpp
Normal 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>>();
|
||||
}
|
||||
|
||||
}
|
20
src/Functions/toUnixTimestamp64Milli.cpp
Normal file
20
src/Functions/toUnixTimestamp64Milli.cpp
Normal 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>>();
|
||||
}
|
||||
|
||||
}
|
20
src/Functions/toUnixTimestamp64Nano.cpp
Normal file
20
src/Functions/toUnixTimestamp64Nano.cpp
Normal 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>>();
|
||||
}
|
||||
|
||||
}
|
@ -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
|
||||
|
@ -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
|
45
tests/queries/0_stateless/01277_fromUnixTimestamp64.sql
Normal file
45
tests/queries/0_stateless/01277_fromUnixTimestamp64.sql
Normal 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;
|
@ -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
|
33
tests/queries/0_stateless/01277_toUnixTimestamp64.sql
Normal file
33
tests/queries/0_stateless/01277_toUnixTimestamp64.sql
Normal 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);
|
||||
|
@ -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
|
@ -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);
|
Loading…
Reference in New Issue
Block a user