From 20cefa3ad65c5323e00b9398248ed4180c9bb73f Mon Sep 17 00:00:00 2001 From: Sasha Sheikin Date: Sun, 6 Oct 2024 01:31:39 +0530 Subject: [PATCH] Support DateTime64 in ClickHouse TIMESTAMP(p) TIMESTAMP(p) WITH TIME ZONE --- docs/src/main/sphinx/connector/clickhouse.md | 9 ++ .../plugin/clickhouse/ClickHouseClient.java | 85 +++++++++++++++- .../clickhouse/BaseClickHouseTypeMapping.java | 99 ++++++++++++------- .../TestClickHouseConnectorTest.java | 2 - 4 files changed, 151 insertions(+), 44 deletions(-) diff --git a/docs/src/main/sphinx/connector/clickhouse.md b/docs/src/main/sphinx/connector/clickhouse.md index 0da7a863bb28..7335ec32eb07 100644 --- a/docs/src/main/sphinx/connector/clickhouse.md +++ b/docs/src/main/sphinx/connector/clickhouse.md @@ -233,6 +233,9 @@ to the following table: * - `DateTime[(timezone)]` - `TIMESTAMP(0) [WITH TIME ZONE]` - +* - `DateTime64[n, (timezone)]` + - `TIMESTAMP(n) [WITH TIME ZONE]` + - `n` in [0-9] * - `IPv4` - `IPADDRESS` - @@ -304,6 +307,12 @@ to the following table: * - `TIMESTAMP(0)` - `DateTime` - +* - `TIMESTAMP(n)` + - `DateTime64(n)` + - `n` in [1-9] +* - `TIMESTAMP(n) WITH TIME ZONE` + - `DateTime64(n, [timezone])` + - `n` in [0-9] * - `UUID` - `UUID` - diff --git a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java index c414d5f6deee..8426e50db7e8 100644 --- a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java +++ b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java @@ -44,6 +44,7 @@ import io.trino.plugin.jdbc.JdbcTypeHandle; import io.trino.plugin.jdbc.LongReadFunction; import io.trino.plugin.jdbc.LongWriteFunction; +import io.trino.plugin.jdbc.ObjectReadFunction; import io.trino.plugin.jdbc.ObjectWriteFunction; import io.trino.plugin.jdbc.QueryBuilder; import io.trino.plugin.jdbc.RemoteTableName; @@ -73,7 +74,11 @@ import io.trino.spi.type.DecimalType; import io.trino.spi.type.Decimals; import io.trino.spi.type.Int128; +import io.trino.spi.type.LongTimestampWithTimeZone; import io.trino.spi.type.StandardTypes; +import io.trino.spi.type.TimeZoneKey; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TimestampWithTimeZoneType; import io.trino.spi.type.Type; import io.trino.spi.type.TypeManager; import io.trino.spi.type.TypeSignature; @@ -145,12 +150,14 @@ import static io.trino.plugin.jdbc.StandardColumnMappings.integerWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalReadFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.longTimestampWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.realWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.shortDecimalWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping; import static io.trino.plugin.jdbc.StandardColumnMappings.smallintWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.timestampColumnMapping; import static io.trino.plugin.jdbc.StandardColumnMappings.timestampReadFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.timestampWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintColumnMapping; import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryColumnMapping; @@ -173,11 +180,15 @@ import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.RealType.REAL; import static io.trino.spi.type.SmallintType.SMALLINT; -import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; import static io.trino.spi.type.TimestampType.TIMESTAMP_SECONDS; +import static io.trino.spi.type.TimestampType.createTimestampType; import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_SECONDS; +import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_SECOND; +import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MICROSECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_NANOSECOND; import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.UuidType.javaUuidToTrinoUuid; import static io.trino.spi.type.UuidType.trinoUuidToJavaUuid; @@ -199,6 +210,7 @@ public class ClickHouseClient extends BaseJdbcClient { + public static final int CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION = 9; private static final Splitter TABLE_PROPERTY_SPLITTER = Splitter.on(',').omitEmptyStrings().trimResults(); private static final DecimalType UINT64_TYPE = createDecimalType(20, 0); @@ -722,9 +734,10 @@ public Optional toColumnMapping(ConnectorSession session, Connect timestampReadFunction(TIMESTAMP_SECONDS), timestampSecondsWriteFunction(getClickHouseServerVersion(session)))); } - // TODO (https://github.com/trinodb/trino/issues/10537) Add support for Datetime64 type - return Optional.of(timestampColumnMapping(TIMESTAMP_MILLIS)); - + if (columnDataType == ClickHouseDataType.DateTime64) { + return Optional.of(timestampColumnMapping(createTimestampType(column.getScale()))); + } + break; case Types.TIMESTAMP_WITH_TIMEZONE: if (columnDataType == ClickHouseDataType.DateTime) { // ClickHouse DateTime does not have sub-second precision @@ -734,6 +747,9 @@ public Optional toColumnMapping(ConnectorSession session, Connect shortTimestampWithTimeZoneReadFunction(), shortTimestampWithTimeZoneWriteFunction(column.getTimeZone()))); } + if (columnDataType == ClickHouseDataType.DateTime64) { + return Optional.of(timestampWithTimeZoneColumnMapping(column)); + } } if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) { @@ -789,12 +805,73 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) if (type == TIMESTAMP_SECONDS) { return WriteMapping.longMapping("DateTime", timestampSecondsWriteFunction(getClickHouseServerVersion(session))); } + if (type instanceof TimestampType timestampType) { + return timestampWriteMapping(timestampType); + } + if (type instanceof TimestampWithTimeZoneType) { + // Clickhouse DateTime64(precision, [timezone]) + // In Clickhouse the time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. + // Timezone agnostic Unix timestamp is stored in tables + // In trino, timezone is not available at the point of time when write mapping is resolved + throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type); + } if (type.equals(uuidType)) { return WriteMapping.sliceMapping("UUID", uuidWriteFunction()); } throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type); } + private WriteMapping timestampWriteMapping(TimestampType type) + { + int precision = type.getPrecision(); + String dataType = "DateTime64(%s)".formatted(precision); + if (type.isShort()) { + return WriteMapping.longMapping(dataType, timestampWriteFunction(createTimestampType(precision))); + } + checkArgument(precision <= CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "Precision is out of range: %s", precision); + return WriteMapping.objectMapping(dataType, longTimestampWriteFunction(type, precision)); + } + + private static ColumnMapping timestampWithTimeZoneColumnMapping(ClickHouseColumn clickHouseColumn) + { + int precision = clickHouseColumn.getScale(); + TimeZone columnTimeZone = clickHouseColumn.getTimeZone(); + checkArgument(precision <= CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "Precision is out of range: %s", precision); + TimestampWithTimeZoneType trinoType = createTimestampWithTimeZoneType(precision); + if (trinoType.isShort()) { + return ColumnMapping.longMapping( + trinoType, + shortTimestampWithTimeZoneReadFunction(), + shortTimestampWithTimeZoneWriteFunction(columnTimeZone)); + } + return ColumnMapping.objectMapping( + trinoType, + longTimestampWithTimeZoneReadFunction(), + longTimestampWithTimeZoneWriteFunction(columnTimeZone)); + } + + private static ObjectReadFunction longTimestampWithTimeZoneReadFunction() + { + return ObjectReadFunction.of(LongTimestampWithTimeZone.class, (resultSet, columnIndex) -> { + ZonedDateTime timestamp = resultSet.getObject(columnIndex, ZonedDateTime.class); + return LongTimestampWithTimeZone.fromEpochSecondsAndFraction( + timestamp.toEpochSecond(), + (long) timestamp.getNano() * PICOSECONDS_PER_NANOSECOND, + TimeZoneKey.getTimeZoneKey(timestamp.getZone().getId())); + }); + } + + private static ObjectWriteFunction longTimestampWithTimeZoneWriteFunction(TimeZone columnTimeZone) + { + return ObjectWriteFunction.of(LongTimestampWithTimeZone.class, (statement, index, value) -> { + long epochMillis = value.getEpochMillis(); + long epochSeconds = Math.floorDiv(epochMillis, MILLISECONDS_PER_SECOND); + long nanos = (long) Math.floorMod(epochMillis, MILLISECONDS_PER_SECOND) * NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND; + Instant instant = Instant.ofEpochSecond(epochSeconds, nanos); + statement.setObject(index, ZonedDateTime.ofInstant(instant, columnTimeZone.toZoneId())); + }); + } + private ClickHouseVersion getClickHouseServerVersion(ConnectorSession session) { return clickHouseVersion.updateAndGet(current -> { diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java index 2444ea4bf6c6..de2cd2cbaa73 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseTypeMapping.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import io.trino.Session; import io.trino.spi.type.TimeZoneKey; +import io.trino.spi.type.TimestampWithTimeZoneType; import io.trino.spi.type.UuidType; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.TestingSession; @@ -36,9 +37,11 @@ import java.time.ZoneId; import java.util.List; import java.util.function.Function; +import java.util.stream.IntStream; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; +import static io.trino.plugin.clickhouse.ClickHouseClient.CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION; import static io.trino.plugin.clickhouse.ClickHouseQueryRunner.TPCH_SCHEMA; import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.UNSUPPORTED_TYPE_HANDLING; import static io.trino.plugin.jdbc.UnsupportedTypeHandling.CONVERT_TO_VARCHAR; @@ -51,6 +54,7 @@ import static io.trino.spi.type.SmallintType.SMALLINT; import static io.trino.spi.type.TimestampType.createTimestampType; import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_SECONDS; +import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -914,6 +918,15 @@ public void testTimestamp() .addRoundTrip("timestamp(0)", "timestamp '2018-03-25 03:17:17'", createTimestampType(0), "TIMESTAMP '2018-03-25 03:17:17'") // time gap in Vilnius .addRoundTrip("timestamp(0)", "timestamp '2018-10-28 01:33:17'", createTimestampType(0), "TIMESTAMP '2018-10-28 01:33:17'") // time doubled in JVM zone .addRoundTrip("timestamp(0)", "timestamp '2018-10-28 03:33:33'", createTimestampType(0), "TIMESTAMP '2018-10-28 03:33:33'") // time double in Vilnius + .addRoundTrip("timestamp(1)", "timestamp '2024-01-01 12:34:56.1'", createTimestampType(1), "TIMESTAMP '2024-01-01 12:34:56.1'") + .addRoundTrip("timestamp(2)", "timestamp '2024-01-01 12:34:56.12'", createTimestampType(2), "TIMESTAMP '2024-01-01 12:34:56.12'") + .addRoundTrip("timestamp(3)", "timestamp '2024-01-01 12:34:56.123'", createTimestampType(3), "TIMESTAMP '2024-01-01 12:34:56.123'") + .addRoundTrip("timestamp(4)", "timestamp '2024-01-01 12:34:56.1234'", createTimestampType(4), "TIMESTAMP '2024-01-01 12:34:56.1234'") + .addRoundTrip("timestamp(5)", "timestamp '2024-01-01 12:34:56.12345'", createTimestampType(5), "TIMESTAMP '2024-01-01 12:34:56.12345'") + .addRoundTrip("timestamp(6)", "timestamp '2024-01-01 12:34:56.123456'", createTimestampType(6), "TIMESTAMP '2024-01-01 12:34:56.123456'") + .addRoundTrip("timestamp(7)", "timestamp '2024-01-01 12:34:56.1234567'", createTimestampType(7), "TIMESTAMP '2024-01-01 12:34:56.1234567'") + .addRoundTrip("timestamp(8)", "timestamp '2024-01-01 12:34:56.12345678'", createTimestampType(8), "TIMESTAMP '2024-01-01 12:34:56.12345678'") + .addRoundTrip("timestamp(9)", "timestamp '2024-01-01 12:34:56.123456789'", createTimestampType(9), "TIMESTAMP '2024-01-01 12:34:56.123456789'") .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAsSelect("test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp")) @@ -1003,62 +1016,77 @@ public void testClickHouseDateTimeWithTimeZone() Session session = Session.builder(getSession()) .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId())) .build(); - SqlDataTypeTest.create() - .addRoundTrip("DateTime('Asia/Kathmandu')", "timestamp '2024-01-01 12:34:56'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2024-01-01 05:19:56 +05:45'") - .addRoundTrip("DateTime('Asia/Kathmandu')", "timestamp '2024-01-01 12:34:56 Asia/Kathmandu'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2024-01-01 12:34:56 +05:45'") - .addRoundTrip("DateTime('Asia/Kathmandu')", "timestamp '2024-01-01 12:34:56 +00:00'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2024-01-01 18:19:56 +05:45'") - .addRoundTrip("DateTime('Asia/Kathmandu')", "timestamp '2024-01-01 12:34:56 -01:00'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2024-01-01 19:19:56 +05:45'") - .execute(getQueryRunner(), session, clickhouseCreateAndTrinoInsert("tpch.test_timestamp_with_time_zone")); - - dateTimeWithTimeZoneTest(clickhouseDateTimeInputTypeFactory("datetime")) - .execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.datetime_tz")); + + SqlDataTypeTest clickhouseCreateAndTrinoInsertTests = SqlDataTypeTest.create() + .addRoundTrip("Nullable(DateTime('UTC'))", "NULL", TIMESTAMP_TZ_SECONDS, "CAST(NULL AS TIMESTAMP(0) WITH TIME ZONE)"); + addTestCasesForClickhouseCreateAndTrinoInsert(clickhouseCreateAndTrinoInsertTests, "DateTime('%s')"::formatted, 0); + IntStream.rangeClosed(0, CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION).forEach(precision -> + addTestCasesForClickhouseCreateAndTrinoInsert(clickhouseCreateAndTrinoInsertTests, zone -> "DateTime64(%s, '%s')".formatted(precision, zone), precision)); + clickhouseCreateAndTrinoInsertTests.execute(getQueryRunner(), session, clickhouseCreateAndTrinoInsert("tpch.test_timestamp_with_time_zone")); + + SqlDataTypeTest clickhouseCreateAndInsertTests = SqlDataTypeTest.create() + .addRoundTrip("Nullable(DateTime('UTC'))", "NULL", TIMESTAMP_TZ_SECONDS, "CAST(NULL AS TIMESTAMP(0) WITH TIME ZONE)"); + addTestCasesClickhouseCreateAndInsert(clickhouseCreateAndInsertTests, "DateTime('%s')"::formatted, 0); + IntStream.rangeClosed(0, CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION).forEach(precision -> + addTestCasesClickhouseCreateAndInsert(clickhouseCreateAndInsertTests, zone -> "DateTime64(%s, '%s')".formatted(precision, zone), precision)); + clickhouseCreateAndInsertTests.execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.datetime_tz")); } } - private SqlDataTypeTest dateTimeWithTimeZoneTest(Function inputTypeFactory) + private void addTestCasesForClickhouseCreateAndTrinoInsert(SqlDataTypeTest tests, Function inputTypeFactory, int precision) { - ZoneId utc = ZoneId.of("UTC"); - SqlDataTypeTest tests = SqlDataTypeTest.create() - .addRoundTrip(format("Nullable(%s)", inputTypeFactory.apply(utc)), "NULL", TIMESTAMP_TZ_SECONDS, "CAST(NULL AS TIMESTAMP(0) WITH TIME ZONE)") + TimestampWithTimeZoneType expectedType = createTimestampWithTimeZoneType(precision); + String zone = "Asia/Kathmandu"; + String nanos = precision == 0 ? "" : "." + "123456789".substring(0, precision); + tests + .addRoundTrip(inputTypeFactory.apply(zone), "timestamp '2024-01-01 12:34:56%s'".formatted(nanos), expectedType, "TIMESTAMP '2024-01-01 05:19:56%s +05:45'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(zone), "timestamp '2024-01-01 12:34:56%s %s'".formatted(nanos, zone), expectedType, "TIMESTAMP '2024-01-01 12:34:56%s +05:45'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(zone), "timestamp '2024-01-01 12:34:56%s +00:00'".formatted(nanos), expectedType, "TIMESTAMP '2024-01-01 18:19:56%s +05:45'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(zone), "timestamp '2024-01-01 12:34:56%s -01:00'".formatted(nanos), expectedType, "TIMESTAMP '2024-01-01 19:19:56%s +05:45'".formatted(nanos)); + } + private void addTestCasesClickhouseCreateAndInsert(SqlDataTypeTest tests, Function inputTypeFactory, int precision) + { + TimestampWithTimeZoneType expectedType = createTimestampWithTimeZoneType(precision); + ZoneId utc = ZoneId.of("UTC"); + String nanos = precision == 0 ? "" : "." + "123456789".substring(0, precision); + tests // Since ClickHouse datetime(timezone) does not support values before epoch, we do not test this here. // epoch - .addRoundTrip(inputTypeFactory.apply(utc), "0", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:00:00 Z'") - .addRoundTrip(inputTypeFactory.apply(utc), "'1970-01-01 00:00:00'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:00:00 Z'") - .addRoundTrip(inputTypeFactory.apply(kathmandu), "'1970-01-01 00:00:00'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 05:30:00 +05:30'") + .addRoundTrip(inputTypeFactory.apply(utc), "0%s".formatted(nanos), expectedType, "TIMESTAMP '1970-01-01 00:00:00%s Z'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(utc), "'1970-01-01 00:00:00%s'".formatted(nanos), expectedType, "TIMESTAMP '1970-01-01 00:00:00%s Z'".formatted(nanos)) + // .addRoundTrip(inputTypeFactory.apply(kathmandu), "'1970-01-01 00:00:00%s'".formatted(nanos), expectedType, "TIMESTAMP '1970-01-01 05:30:00%s +05:30'".formatted(nanos)) // after epoch - .addRoundTrip(inputTypeFactory.apply(utc), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 Z'") - .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 +05:45'") - .addRoundTrip(inputTypeFactory.apply(ZoneId.of("GMT")), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 Z'") - .addRoundTrip(inputTypeFactory.apply(ZoneId.of("UTC+00:00")), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 Z'") + .addRoundTrip(inputTypeFactory.apply(utc), "'2019-03-18 10:01:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2019-03-18 10:01:17%s Z'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2019-03-18 10:01:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2019-03-18 10:01:17%s +05:45'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(ZoneId.of("GMT")), "'2019-03-18 10:01:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2019-03-18 10:01:17%s Z'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(ZoneId.of("UTC+00:00")), "'2019-03-18 10:01:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2019-03-18 10:01:17%s Z'".formatted(nanos)) // time doubled in JVM zone - .addRoundTrip(inputTypeFactory.apply(utc), "'2018-10-28 01:33:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 01:33:17 Z'") - .addRoundTrip(inputTypeFactory.apply(jvmZone), "'2018-10-28 01:33:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 01:33:17 -05:00'") - .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-10-28 01:33:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 01:33:17 +05:45'") + .addRoundTrip(inputTypeFactory.apply(utc), "'2018-10-28 01:33:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-10-28 01:33:17%s Z'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(jvmZone), "'2018-10-28 01:33:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-10-28 01:33:17%s -05:00'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-10-28 01:33:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-10-28 01:33:17%s +05:45'".formatted(nanos)) // time doubled in Vilnius - .addRoundTrip(inputTypeFactory.apply(utc), "'2018-10-28 03:33:33'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 03:33:33 Z'") - .addRoundTrip(inputTypeFactory.apply(vilnius), "'2018-10-28 03:33:33'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 03:33:33 +03:00'") - .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-10-28 03:33:33'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 03:33:33 +05:45'") + .addRoundTrip(inputTypeFactory.apply(utc), "'2018-10-28 03:33:33%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-10-28 03:33:33%s Z'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(vilnius), "'2018-10-28 03:33:33%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-10-28 03:33:33%s +03:00'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-10-28 03:33:33%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-10-28 03:33:33%s +05:45'".formatted(nanos)) // time gap in JVM zone - .addRoundTrip(inputTypeFactory.apply(utc), "'1970-01-01 00:13:42'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:13:42 Z'") + .addRoundTrip(inputTypeFactory.apply(utc), "'1970-01-01 00:13:42%s'".formatted(nanos), expectedType, "TIMESTAMP '1970-01-01 00:13:42%s Z'".formatted(nanos)) // TODO: Check the range of DateTime(timezone) values written from Trino to ClickHouse to prevent ClickHouse from storing incorrect results. // e.g. 1970-01-01 00:13:42 will become 1970-01-01 05:30:00 // .addRoundTrip(inputTypeFactory.apply(kathmandu), "'1970-01-01 00:13:42'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:13:42 +05:30'") - .addRoundTrip(inputTypeFactory.apply(utc), "'2018-04-01 02:13:55'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-04-01 02:13:55 Z'") - .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-04-01 02:13:55'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-04-01 02:13:55 +05:45'") + .addRoundTrip(inputTypeFactory.apply(utc), "'2018-04-01 02:13:55%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-04-01 02:13:55%s Z'".formatted(nanos)) + .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-04-01 02:13:55%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-04-01 02:13:55%s +05:45'".formatted(nanos)) // time gap in Vilnius - .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-03-25 03:17:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-03-25 03:17:17 +05:45'") + .addRoundTrip(inputTypeFactory.apply(kathmandu), "'2018-03-25 03:17:17%s'".formatted(nanos), expectedType, "TIMESTAMP '2018-03-25 03:17:17%s +05:45'".formatted(nanos)) // time gap in Kathmandu - .addRoundTrip(inputTypeFactory.apply(vilnius), "'1986-01-01 00:13:07'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1986-01-01 00:13:07 +03:00'"); - - return tests; + .addRoundTrip(inputTypeFactory.apply(vilnius), "'1986-01-01 00:13:07%s'".formatted(nanos), expectedType, "TIMESTAMP '1986-01-01 00:13:07%s +03:00'".formatted(nanos)); } private List timezones() @@ -1176,9 +1204,4 @@ protected SqlExecutor onRemoteDatabase() { return clickhouseServer::execute; } - - private static Function clickhouseDateTimeInputTypeFactory(String inputTypePrefix) - { - return zone -> format("%s('%s')", inputTypePrefix, zone); - } } diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 565ecba0fbe0..7f1982928d2f 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -601,8 +601,6 @@ protected Optional filterDataMappingSmokeTestData(DataMapp case "time": case "time(6)": - case "timestamp": - case "timestamp(6)": case "timestamp(3) with time zone": case "timestamp(6) with time zone": return Optional.of(dataMappingTestSetup.asUnsupported());