mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Support from/to utc timestamp
This commit is contained in:
parent
b074e44c1b
commit
48cb9b59f4
@ -1819,6 +1819,72 @@ Result:
|
|||||||
└────────────────────────────────────┘
|
└────────────────────────────────────┘
|
||||||
```
|
```
|
||||||
|
|
||||||
|
## toUTCTimestamp
|
||||||
|
|
||||||
|
Convert DateTime/DateTime64 type value from other time zone to UTC timezone timestamp
|
||||||
|
|
||||||
|
**Syntax**
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
toUTCTimestamp(time_val, time_zone)
|
||||||
|
```
|
||||||
|
|
||||||
|
**Arguments**
|
||||||
|
|
||||||
|
- `time_val` — A DateTime/DateTime64 type const value or a expression . [DateTime/DateTime64 types](../../sql-reference/data-types/datetime.md)
|
||||||
|
- `time_zone` — A String type const value or a expression represent the time zone. [String types](../../sql-reference/data-types/string.md)
|
||||||
|
|
||||||
|
**Returned value**
|
||||||
|
|
||||||
|
- DateTime/DateTime64 in text form
|
||||||
|
|
||||||
|
**Example**
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
SELECT toUTCTimestamp(toDateTime('2023-03-16'), 'Asia/Shanghai');
|
||||||
|
```
|
||||||
|
|
||||||
|
Result:
|
||||||
|
|
||||||
|
``` text
|
||||||
|
┌─toUTCTimestamp(toDateTime('2023-03-16'),'Asia/Shanghai')┐
|
||||||
|
│ 2023-03-15 16:00:00 │
|
||||||
|
└─────────────────────────────────────────────────────────┘
|
||||||
|
```
|
||||||
|
|
||||||
|
## fromUTCTimestamp
|
||||||
|
|
||||||
|
Convert DateTime/DateTime64 type value from UTC timezone to other time zone timestamp
|
||||||
|
|
||||||
|
**Syntax**
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
fromUTCTimestamp(time_val, time_zone)
|
||||||
|
```
|
||||||
|
|
||||||
|
**Arguments**
|
||||||
|
|
||||||
|
- `time_val` — A DateTime/DateTime64 type const value or a expression . [DateTime/DateTime64 types](../../sql-reference/data-types/datetime.md)
|
||||||
|
- `time_zone` — A String type const value or a expression represent the time zone. [String types](../../sql-reference/data-types/string.md)
|
||||||
|
|
||||||
|
**Returned value**
|
||||||
|
|
||||||
|
- DateTime/DateTime64 in text form
|
||||||
|
|
||||||
|
**Example**
|
||||||
|
|
||||||
|
``` sql
|
||||||
|
SELECT fromUTCTimestamp(toDateTime64('2023-03-16 10:00:00', 3), 'Asia/Shanghai');
|
||||||
|
```
|
||||||
|
|
||||||
|
Result:
|
||||||
|
|
||||||
|
``` text
|
||||||
|
┌─fromUTCTimestamp(toDateTime64('2023-03-16 10:00:00',3),'Asia/Shanghai')─┐
|
||||||
|
│ 2023-03-16 18:00:00.000 │
|
||||||
|
└─────────────────────────────────────────────────────────────────────────┘
|
||||||
|
```
|
||||||
|
|
||||||
## Related content
|
## Related content
|
||||||
|
|
||||||
- Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse)
|
- Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse)
|
||||||
|
141
src/Functions/UTCTimestampTransform.cpp
Normal file
141
src/Functions/UTCTimestampTransform.cpp
Normal file
@ -0,0 +1,141 @@
|
|||||||
|
#include <Columns/ColumnConst.h>
|
||||||
|
#include <Columns/ColumnsDateTime.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnsNumber.h>
|
||||||
|
#include <Columns/IColumn.h>
|
||||||
|
#include <Common/DateLUT.h>
|
||||||
|
#include <Common/LocalDateTime.h>
|
||||||
|
#include <Common/logger_useful.h>
|
||||||
|
#include <Core/DecimalFunctions.h>
|
||||||
|
#include <DataTypes/DataTypeDateTime.h>
|
||||||
|
#include <DataTypes/DataTypeDateTime64.h>
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DataTypes/TimezoneMixin.h>
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <Functions/IFunction.h>
|
||||||
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
template <typename Name>
|
||||||
|
class UTCTimestampTransform : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static FunctionPtr create(ContextPtr) { return std::make_shared<UTCTimestampTransform>(); }
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
|
||||||
|
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||||
|
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
if (arguments.size() != 2)
|
||||||
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {}'s arguments number must be 2.", name);
|
||||||
|
WhichDataType which_type_first(arguments[0]);
|
||||||
|
WhichDataType which_type_second(arguments[1]);
|
||||||
|
if (!which_type_first.isDateTime() && !which_type_first.isDateTime64())
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s 1st argument type must be datetime.", name);
|
||||||
|
if (dynamic_cast<const TimezoneMixin *>(arguments[0].get())->hasExplicitTimeZone())
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s 1st argument should not have explicit time zone.", name);
|
||||||
|
if (!which_type_second.isString())
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s 2nd argument type must be string.", name);
|
||||||
|
DataTypePtr date_time_type;
|
||||||
|
if (which_type_first.isDateTime())
|
||||||
|
date_time_type = std::make_shared<DataTypeDateTime>();
|
||||||
|
else
|
||||||
|
{
|
||||||
|
const DataTypeDateTime64 * date_time_64 = static_cast<const DataTypeDateTime64 *>(arguments[0].get());
|
||||||
|
date_time_type = std::make_shared<DataTypeDateTime64>(date_time_64->getScale());
|
||||||
|
}
|
||||||
|
return date_time_type;
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override
|
||||||
|
{
|
||||||
|
if (arguments.size() != 2)
|
||||||
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {}'s arguments number must be 2.", name);
|
||||||
|
ColumnWithTypeAndName arg1 = arguments[0];
|
||||||
|
ColumnWithTypeAndName arg2 = arguments[1];
|
||||||
|
const auto * time_zone_const_col = checkAndGetColumnConstData<ColumnString>(arg2.column.get());
|
||||||
|
if (!time_zone_const_col)
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of 2nd argument of function {}. Excepted const(String).", arg2.column->getName(), name);
|
||||||
|
String time_zone_val = time_zone_const_col->getDataAt(0).toString();
|
||||||
|
auto column = result_type->createColumn();
|
||||||
|
if (WhichDataType(arg1.type).isDateTime())
|
||||||
|
{
|
||||||
|
const auto * date_time_col = checkAndGetColumn<ColumnDateTime>(arg1.column.get());
|
||||||
|
for (size_t i = 0; i < date_time_col->size(); ++i)
|
||||||
|
{
|
||||||
|
UInt32 date_time_val = date_time_col->getElement(i);
|
||||||
|
LocalDateTime date_time(date_time_val, Name::to ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
|
||||||
|
time_t time_val = date_time.to_time_t(Name::from ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
|
||||||
|
column->insert(time_val);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if (WhichDataType(arg1.type).isDateTime64())
|
||||||
|
{
|
||||||
|
const auto * date_time_col = checkAndGetColumn<ColumnDateTime64>(arg1.column.get());
|
||||||
|
const DataTypeDateTime64 * date_time_type = static_cast<const DataTypeDateTime64 *>(arg1.type.get());
|
||||||
|
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(date_time_type->getScale());
|
||||||
|
for (size_t i = 0; i < date_time_col->size(); ++i)
|
||||||
|
{
|
||||||
|
DateTime64 date_time_val = date_time_col->getElement(i);
|
||||||
|
Int64 seconds = date_time_val.value / scale_multiplier;
|
||||||
|
Int64 micros = date_time_val.value % scale_multiplier;
|
||||||
|
LocalDateTime date_time(seconds, Name::to ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
|
||||||
|
time_t time_val = date_time.to_time_t(Name::from ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
|
||||||
|
DateTime64 date_time_64(time_val * scale_multiplier + micros);
|
||||||
|
column->insert(date_time_64);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s 1st argument can only be datetime/datatime64. ", name);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
|
||||||
|
};
|
||||||
|
|
||||||
|
struct NameToUTCTimestamp
|
||||||
|
{
|
||||||
|
static constexpr auto name = "toUTCTimestamp";
|
||||||
|
static constexpr auto from = false;
|
||||||
|
static constexpr auto to = true;
|
||||||
|
};
|
||||||
|
|
||||||
|
struct NameFromUTCTimestamp
|
||||||
|
{
|
||||||
|
static constexpr auto name = "fromUTCTimestamp";
|
||||||
|
static constexpr auto from = true;
|
||||||
|
static constexpr auto to = false;
|
||||||
|
};
|
||||||
|
|
||||||
|
using ToUTCTimestampFunction = UTCTimestampTransform<NameToUTCTimestamp>;
|
||||||
|
using FromUTCTimestampFunction = UTCTimestampTransform<NameFromUTCTimestamp>;
|
||||||
|
}
|
||||||
|
|
||||||
|
REGISTER_FUNCTION(UTCTimestampTransform)
|
||||||
|
{
|
||||||
|
factory.registerFunction<ToUTCTimestampFunction>();
|
||||||
|
factory.registerFunction<FromUTCTimestampFunction>();
|
||||||
|
factory.registerAlias("to_utc_timestamp", NameToUTCTimestamp::name, FunctionFactory::CaseInsensitive);
|
||||||
|
factory.registerAlias("from_utc_timestamp", NameFromUTCTimestamp::name, FunctionFactory::CaseInsensitive);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -302,6 +302,7 @@ formatRowNoNewline
|
|||||||
fragment
|
fragment
|
||||||
fromModifiedJulianDay
|
fromModifiedJulianDay
|
||||||
fromModifiedJulianDayOrNull
|
fromModifiedJulianDayOrNull
|
||||||
|
fromUTCTimestamp
|
||||||
fromUnixTimestamp
|
fromUnixTimestamp
|
||||||
fromUnixTimestamp64Micro
|
fromUnixTimestamp64Micro
|
||||||
fromUnixTimestamp64Milli
|
fromUnixTimestamp64Milli
|
||||||
@ -849,6 +850,7 @@ toUInt8
|
|||||||
toUInt8OrDefault
|
toUInt8OrDefault
|
||||||
toUInt8OrNull
|
toUInt8OrNull
|
||||||
toUInt8OrZero
|
toUInt8OrZero
|
||||||
|
toUTCTimestamp
|
||||||
toUUID
|
toUUID
|
||||||
toUUIDOrDefault
|
toUUIDOrDefault
|
||||||
toUUIDOrNull
|
toUUIDOrNull
|
||||||
|
@ -0,0 +1,3 @@
|
|||||||
|
1 2023-03-16 12:22:33 2023-03-16 10:22:33.000 2023-03-15 16:00:00 2023-03-16 19:22:33.000
|
||||||
|
2 2023-03-16 12:22:33 2023-03-16 10:22:33.000 2023-03-16 03:22:33 2023-03-16 08:00:00.000
|
||||||
|
3 2023-03-16 12:22:33 2023-03-16 10:22:33.000 2023-03-16 03:22:33 2023-03-16 19:22:33.123
|
15
tests/queries/0_stateless/02812_from_to_utc_timestamp.sh
Executable file
15
tests/queries/0_stateless/02812_from_to_utc_timestamp.sh
Executable file
@ -0,0 +1,15 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
# NOTE: this sh wrapper is required because of shell_config
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CURDIR"/../shell_config.sh
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl"
|
||||||
|
$CLICKHOUSE_CLIENT -q "create table test_tbl (x UInt32, y DateTime, z DateTime64) engine=MergeTree order by x"
|
||||||
|
${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl values(1, '2023-03-16', '2023-03-16 11:22:33')"
|
||||||
|
${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl values(2, '2023-03-16 11:22:33', '2023-03-16')"
|
||||||
|
${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl values(3, '2023-03-16 11:22:33', '2023-03-16 11:22:33.123456')"
|
||||||
|
$CLICKHOUSE_CLIENT -q "select x, to_utc_timestamp(toDateTime('2023-03-16 11:22:33'), 'Etc/GMT+1'), from_utc_timestamp(toDateTime64('2023-03-16 11:22:33', 3), 'Etc/GMT+1'), to_utc_timestamp(y, 'Asia/Shanghai'), from_utc_timestamp(z, 'Asia/Shanghai') from test_tbl order by x"
|
||||||
|
$CLICKHOUSE_CLIENT -q "drop table test_tbl"
|
@ -1464,6 +1464,7 @@ formatter
|
|||||||
freezed
|
freezed
|
||||||
fromModifiedJulianDay
|
fromModifiedJulianDay
|
||||||
fromModifiedJulianDayOrNull
|
fromModifiedJulianDayOrNull
|
||||||
|
fromUTCTimestamp
|
||||||
fromUnixTimestamp
|
fromUnixTimestamp
|
||||||
fromUnixTimestampInJodaSyntax
|
fromUnixTimestampInJodaSyntax
|
||||||
fsync
|
fsync
|
||||||
@ -2394,6 +2395,7 @@ toTimeZone
|
|||||||
toType
|
toType
|
||||||
toTypeName
|
toTypeName
|
||||||
toUInt
|
toUInt
|
||||||
|
toUTCTimestamp
|
||||||
toUUID
|
toUUID
|
||||||
toUUIDOrDefault
|
toUUIDOrDefault
|
||||||
toUUIDOrNull
|
toUUIDOrNull
|
||||||
|
Loading…
Reference in New Issue
Block a user