diff --git a/docs/src/main/sphinx/connector/clickhouse.md b/docs/src/main/sphinx/connector/clickhouse.md index 4af9cb75ddf8..0e3fb0c0ab9b 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(p[, timezone])` + - `TIMESTAMP(p) [WITH TIME ZONE]` + - * - `IPv4` - `IPADDRESS` - @@ -304,6 +307,9 @@ to the following table: * - `TIMESTAMP(0)` - `DateTime` - +* - `TIMESTAMP(p)` + - `DateTime64(p)` + - * - `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 515ff2608117..01477b72e8cf 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; @@ -74,7 +75,11 @@ import io.trino.spi.type.DecimalType; import io.trino.spi.type.Decimals; import io.trino.spi.type.Int128; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.LongTimestampWithTimeZone; import io.trino.spi.type.StandardTypes; +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; @@ -97,6 +102,7 @@ import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.Collection; import java.util.List; @@ -124,6 +130,7 @@ import static io.trino.plugin.clickhouse.ClickHouseTableProperties.PRIMARY_KEY_PROPERTY; import static io.trino.plugin.clickhouse.ClickHouseTableProperties.SAMPLE_BY_PROPERTY; import static io.trino.plugin.clickhouse.TrinoToClickHouseWriteChecker.DATETIME; +import static io.trino.plugin.clickhouse.TrinoToClickHouseWriteChecker.DATETIME64; import static io.trino.plugin.clickhouse.TrinoToClickHouseWriteChecker.UINT16; import static io.trino.plugin.clickhouse.TrinoToClickHouseWriteChecker.UINT32; import static io.trino.plugin.clickhouse.TrinoToClickHouseWriteChecker.UINT64; @@ -147,6 +154,7 @@ 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.longTimestampReadFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.realWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.shortDecimalWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping; @@ -175,11 +183,18 @@ 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.TimeZoneKey.getTimeZoneKey; 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.Timestamps.round; import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.UuidType.javaUuidToTrinoUuid; import static io.trino.spi.type.UuidType.trinoUuidToJavaUuid; @@ -188,6 +203,7 @@ import static java.lang.Math.floorDiv; import static java.lang.Math.floorMod; import static java.lang.Math.max; +import static java.lang.Math.min; import static java.lang.Math.toIntExact; import static java.lang.String.format; import static java.lang.String.join; @@ -209,6 +225,7 @@ public class ClickHouseClient private static final String NO_COMMENT = ""; public static final int DEFAULT_DOMAIN_COMPACTION_THRESHOLD = 1_000; + public static final int CLICKHOUSE_MAX_SUPPORTED_DATETIME64_PRECISION = 9; private final ConnectorExpressionRewriter connectorExpressionRewriter; private final AggregateFunctionRewriter aggregateFunctionRewriter; @@ -711,7 +728,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect ColumnMapping decimalColumnMapping; if (getDecimalRounding(session) == ALLOW_OVERFLOW && precision > Decimals.MAX_PRECISION) { - int scale = Math.min(decimalDigits, getDecimalDefaultScale(session)); + int scale = min(decimalDigits, getDecimalDefaultScale(session)); decimalColumnMapping = decimalColumnMapping(createDecimalType(Decimals.MAX_PRECISION, scale), getDecimalRoundingMode(session)); } else { @@ -736,8 +753,11 @@ public Optional toColumnMapping(ConnectorSession session, Connect timestampReadFunction(TIMESTAMP_SECONDS), timestampSecondsWriteFunction(version))); } - // 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(typeHandle.requiredDecimalDigits()), getClickHouseServerVersion(session))); + } + // TODO Add support for Datetime32 type + return Optional.of(timestampColumnMapping(TIMESTAMP_MILLIS, version)); case Types.TIMESTAMP_WITH_TIMEZONE: if (columnDataType == ClickHouseDataType.DateTime) { @@ -746,7 +766,10 @@ public Optional toColumnMapping(ConnectorSession session, Connect return Optional.of(ColumnMapping.longMapping( TIMESTAMP_TZ_SECONDS, shortTimestampWithTimeZoneReadFunction(), - shortTimestampWithTimeZoneWriteFunction(version, column.getTimeZone()))); + shortTimestampWithTimeZoneWriteFunction(DATETIME, version, column.getTimeZone()))); + } + if (columnDataType == ClickHouseDataType.DateTime64) { + return Optional.of(timestampWithTimeZoneColumnMapping(column, typeHandle.requiredDecimalDigits(), version)); } } @@ -799,8 +822,12 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) if (type == DATE) { return WriteMapping.longMapping("Date", dateWriteFunctionUsingLocalDate(getClickHouseServerVersion(session))); } - if (type == TIMESTAMP_SECONDS) { - return WriteMapping.longMapping("DateTime", timestampSecondsWriteFunction(getClickHouseServerVersion(session))); + if (type instanceof TimestampType timestampType) { + int precision = min(timestampType.getPrecision(), CLICKHOUSE_MAX_SUPPORTED_DATETIME64_PRECISION); + if (precision <= TimestampType.MAX_SHORT_PRECISION) { + return WriteMapping.longMapping(format("DateTime64(%d)", precision), shortTimestampWriteFunction(getClickHouseServerVersion(session))); + } + return WriteMapping.objectMapping(format("DateTime64(%d)", precision), longTimestampWriteFunction(getClickHouseServerVersion(session), precision)); } if (type.equals(uuidType)) { return WriteMapping.sliceMapping("UUID", uuidWriteFunction()); @@ -906,6 +933,21 @@ private static LongWriteFunction dateWriteFunctionUsingLocalDate(ClickHouseVersi }; } + private static ColumnMapping timestampColumnMapping(TimestampType timestampType, ClickHouseVersion version) + { + if (timestampType.getPrecision() <= TimestampType.MAX_SHORT_PRECISION) { + return ColumnMapping.longMapping( + timestampType, + timestampReadFunction(timestampType), + shortTimestampWriteFunction(version)); + } + checkArgument(timestampType.getPrecision() <= CLICKHOUSE_MAX_SUPPORTED_DATETIME64_PRECISION, "Precision is out of range: %s", timestampType.getPrecision()); + return ColumnMapping.objectMapping( + timestampType, + longTimestampReadFunction(timestampType), + longTimestampWriteFunction(version, timestampType.getPrecision())); + } + private static LongWriteFunction timestampSecondsWriteFunction(ClickHouseVersion version) { return (statement, index, value) -> { @@ -919,6 +961,52 @@ private static LongWriteFunction timestampSecondsWriteFunction(ClickHouseVersion }; } + private static LongWriteFunction shortTimestampWriteFunction(ClickHouseVersion version) + { + return (statement, index, value) -> { + long epochSecond = floorDiv(value, MICROSECONDS_PER_SECOND); + int nanoFraction = floorMod(value, MICROSECONDS_PER_SECOND) * NANOSECONDS_PER_MICROSECOND; + Instant instant = Instant.ofEpochSecond(epochSecond, nanoFraction); + LocalDateTime timestamp = LocalDateTime.ofInstant(instant, UTC); + // ClickHouse stores incorrect results when the values are out of supported range. + DATETIME64.validate(version, timestamp); + statement.setObject(index, timestamp); + }; + } + + private static ObjectWriteFunction longTimestampWriteFunction(ClickHouseVersion version, int precision) + { + checkArgument(precision > TimestampType.MAX_SHORT_PRECISION && precision <= CLICKHOUSE_MAX_SUPPORTED_DATETIME64_PRECISION, + "Precision is out of range: %s", precision); + return ObjectWriteFunction.of( + LongTimestamp.class, + (statement, index, value) -> { + long epochSeconds = floorDiv(value.getEpochMicros(), MICROSECONDS_PER_SECOND); + int microsOfSecond = floorMod(value.getEpochMicros(), MICROSECONDS_PER_SECOND); + long picosOfMicro = round(value.getPicosOfMicro(), TimestampType.MAX_PRECISION - precision); + int nanosOfSecond = (microsOfSecond * NANOSECONDS_PER_MICROSECOND) + toIntExact(picosOfMicro / PICOSECONDS_PER_NANOSECOND); + LocalDateTime timestamp = LocalDateTime.ofInstant(Instant.ofEpochSecond(epochSeconds, nanosOfSecond), UTC); + // ClickHouse stores incorrect results when the values are out of supported range. + DATETIME64.validate(version, timestamp); + statement.setObject(index, timestamp); + }); + } + + private static ColumnMapping timestampWithTimeZoneColumnMapping(ClickHouseColumn column, int precision, ClickHouseVersion version) + { + checkArgument(precision <= CLICKHOUSE_MAX_SUPPORTED_DATETIME64_PRECISION, "Unsupported precision %s", precision); + if (precision <= TimestampWithTimeZoneType.MAX_SHORT_PRECISION) { + return ColumnMapping.longMapping( + createTimestampWithTimeZoneType(precision), + shortTimestampWithTimeZoneReadFunction(), + shortTimestampWithTimeZoneWriteFunction(DATETIME64, version, column.getTimeZone())); + } + return ColumnMapping.objectMapping( + createTimestampWithTimeZoneType(precision), + longTimestampWithTimeZoneReadFunction(), + longTimestampWithTimeZoneWriteFunction()); + } + private static LongReadFunction shortTimestampWithTimeZoneReadFunction() { return (resultSet, columnIndex) -> { @@ -927,18 +1015,44 @@ private static LongReadFunction shortTimestampWithTimeZoneReadFunction() }; } - private static LongWriteFunction shortTimestampWithTimeZoneWriteFunction(ClickHouseVersion version, TimeZone columnTimeZone) + private static LongWriteFunction shortTimestampWithTimeZoneWriteFunction(TrinoToClickHouseWriteChecker valueChecker, ClickHouseVersion version, TimeZone columnTimeZone) { return (statement, index, value) -> { long millisUtc = unpackMillisUtc(value); // Clickhouse JDBC driver inserts datetime as string value as yyyy-MM-dd HH:mm:ss and zone from the Column metadata would be used. Instant instant = Instant.ofEpochMilli(millisUtc); // ClickHouse stores incorrect results when the values are out of supported range. - DATETIME.validate(version, instant.atZone(UTC).toLocalDateTime()); + valueChecker.validate(version, instant.atZone(UTC).toLocalDateTime()); statement.setObject(index, instant.atZone(columnTimeZone.toZoneId())); }; } + private static ObjectReadFunction longTimestampWithTimeZoneReadFunction() + { + return ObjectReadFunction.of( + LongTimestampWithTimeZone.class, + (resultSet, columnIndex) -> { + ZonedDateTime zonedDateTime = resultSet.getObject(columnIndex, ZonedDateTime.class); + return LongTimestampWithTimeZone.fromEpochSecondsAndFraction( + zonedDateTime.toEpochSecond(), + (long) zonedDateTime.getNano() * PICOSECONDS_PER_NANOSECOND, + getTimeZoneKey(zonedDateTime.getZone().getId())); + }); + } + + private static ObjectWriteFunction longTimestampWithTimeZoneWriteFunction() + { + return ObjectWriteFunction.of( + LongTimestampWithTimeZone.class, + (statement, index, value) -> { + long epochMillis = value.getEpochMillis(); + long epochSeconds = floorDiv(epochMillis, MILLISECONDS_PER_SECOND); + int nanoAdjustment = floorMod(epochMillis, MILLISECONDS_PER_SECOND) * NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND; + ZoneId zoneId = getTimeZoneKey(value.getTimeZoneKey()).getZoneId(); + statement.setObject(index, Instant.ofEpochSecond(epochSeconds, nanoAdjustment).atZone(zoneId)); + }); + } + private ColumnMapping ipAddressColumnMapping(String clickhouseType) { return ColumnMapping.sliceMapping( diff --git a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/TrinoToClickHouseWriteChecker.java b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/TrinoToClickHouseWriteChecker.java index e5cf8ef5f21d..4929119877d4 100644 --- a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/TrinoToClickHouseWriteChecker.java +++ b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/TrinoToClickHouseWriteChecker.java @@ -36,16 +36,18 @@ public class TrinoToClickHouseWriteChecker // Different versions of ClickHouse may support different min/max values for the // same data type, you can refer to the table below: // - // | version | column type | min value | max value | - // |---------|-------------|---------------------|----------------------| - // | any | UInt8 | 0 | 255 | - // | any | UInt16 | 0 | 65535 | - // | any | UInt32 | 0 | 4294967295 | - // | any | UInt64 | 0 | 18446744073709551615 | - // | < 21.4 | Date | 1970-01-01 | 2106-02-07 | - // | < 21.4 | DateTime | 1970-01-01 00:00:00 | 2106-02-06 06:28:15 | - // | >= 21.4 | Date | 1970-01-01 | 2149-06-06 | - // | >= 21.4 | DateTime | 1970-01-01 00:00:00 | 2106-02-07 06:28:15 | + // | version | column type | min value | max value | + // |---------+-------------+---------------------+-------------------------------| + // | any | UInt8 | 0 | 255 | + // | any | UInt16 | 0 | 65535 | + // | any | UInt32 | 0 | 4294967295 | + // | any | UInt64 | 0 | 18446744073709551615 | + // | < 21.4 | Date | 1970-01-01 | 2106-02-07 | + // | < 21.4 | DateTime | 1970-01-01 00:00:00 | 2106-02-06 06:28:15 | + // | < 21.4 | DateTime64 | 1970-01-01 00:00:00 | 2106-02-06 06:28:15.999999999 | + // | >= 21.4 | Date | 1970-01-01 | 2149-06-06 | + // | >= 21.4 | DateTime | 1970-01-01 00:00:00 | 2106-02-07 06:28:15 | + // | >= 21.4 | DateTime64 | 1925-01-01 00:00:00 | 2283-11-11 23:59:59.99999999 | // // And when the value written to ClickHouse is out of range, ClickHouse will store // the incorrect result, so we need to check the range of the written value to @@ -68,6 +70,14 @@ public class TrinoToClickHouseWriteChecker new TimestampWriteValueChecker( version -> version.isNewerOrEqualTo("21.4"), new Range<>(LocalDateTime.parse("1970-01-01T00:00:00"), LocalDateTime.parse("2106-02-07T06:28:15"))))); + public static final TrinoToClickHouseWriteChecker DATETIME64 = new TrinoToClickHouseWriteChecker<>( + ImmutableList.of( + new TimestampWriteValueChecker( + version -> version.isOlderThan("21.4"), + new Range<>(LocalDateTime.parse("1900-01-01T00:00:00"), LocalDateTime.parse("2262-04-11T23:47:16"))), + new TimestampWriteValueChecker( + version -> version.isNewerOrEqualTo("21.4"), + new Range<>(LocalDateTime.parse("1900-01-01T00:00:00"), LocalDateTime.parse("2262-04-11T23:47:16"))))); private final List> checkers; 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 d42a6d9024b4..35045bd6e9db 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 @@ -35,10 +35,13 @@ import java.time.LocalDateTime; import java.time.ZoneId; import java.util.List; +import java.util.function.BiFunction; 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_DATETIME64_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; @@ -52,6 +55,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; @@ -59,6 +63,7 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static io.trino.type.IpAddressType.IPADDRESS; import static java.lang.String.format; +import static java.util.stream.Collectors.joining; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -72,11 +77,16 @@ public abstract class BaseClickHouseTypeMapping // minutes offset change since 1932-04-01, no DST private static final ZoneId KATHMANDU = ZoneId.of("Asia/Kathmandu"); private static final Function DATETIME_TYPE_FACTORY = "DateTime('%s')"::formatted; + private static final BiFunction DATETIME64_TYPE_FACTORY = (precision, zone) -> format("DateTime64(%d, '%s')", precision, zone); // https://clickhouse.com/docs/sql-reference/data-types/datetime private static final String MIN_SUPPORTED_DATETIME_VALUE = "1970-01-01 00:00:00"; private static final String MAX_SUPPORTED_DATETIME_VALUE = "2106-02-07 06:28:15"; + // https://clickhouse.com/docs/sql-reference/data-types/datetime64 + private static final String MIN_SUPPORTED_DATETIME64_VALUE = "1900-01-01 00:00:00"; + private static final String MAX_SUPPORTED_DATETIME64_VALUE = "2262-04-11 23:47:16"; + protected TestingClickHouseServer clickhouseServer; @BeforeAll @@ -935,16 +945,31 @@ public void testTimestamp() .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId())) .build(); - SqlDataTypeTest.create() - .addRoundTrip("timestamp(0)", "timestamp '1986-01-01 00:13:07'", createTimestampType(0), "TIMESTAMP '1986-01-01 00:13:07'") // time gap in Kathmandu - .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 - .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")) + SqlDataTypeTest timestampTests = SqlDataTypeTest.create(); + IntStream.rangeClosed(0, CLICKHOUSE_MAX_SUPPORTED_DATETIME64_PRECISION).forEach(precision -> { + String fractionalSeconds = IntStream.rangeClosed(1, precision).mapToObj(String::valueOf).collect(joining("", precision == 0 ? "" : ".", "")); + timestampTests.addRoundTrip(format("timestamp(%d)", precision), format("timestamp '1986-01-01 00:13:07%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '1986-01-01 00:13:07%s'", fractionalSeconds)) // time gap in Kathmandu + .addRoundTrip(format("timestamp(%d)", precision), format("timestamp '2018-03-25 03:17:17%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '2018-03-25 03:17:17%s'", fractionalSeconds)) // time gap in Vilnius + .addRoundTrip(format("timestamp(%d)", precision), format("timestamp '2018-10-28 01:33:17%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '2018-10-28 01:33:17%s'", fractionalSeconds)) // time doubled in JVM zone + .addRoundTrip(format("timestamp(%d)", precision), format("timestamp '2018-10-28 03:33:33%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '2018-10-28 03:33:33%s'", fractionalSeconds)); // time double in Vilnius + }); + + timestampTests.execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAsSelect("test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAndInsert("test_timestamp")); + SqlDataTypeTest datetime64Tests = SqlDataTypeTest.create(); + IntStream.rangeClosed(0, 9).forEach(precision -> { + String fractionalSeconds = IntStream.rangeClosed(1, precision).mapToObj(String::valueOf).collect(joining("", precision == 0 ? "" : ".", "")); + datetime64Tests.addRoundTrip(format("datetime64(%d)", precision), format("'1986-01-01 00:13:07%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '1986-01-01 00:13:07%s'", fractionalSeconds)) // time gap in Kathmandu + .addRoundTrip(format("datetime64(%d)", precision), format("'2018-03-25 03:17:17%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '2018-03-25 03:17:17%s'", fractionalSeconds)) // time gap in Vilnius + .addRoundTrip(format("datetime64(%d)", precision), format("'2018-10-28 01:33:17%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '2018-10-28 01:33:17%s'", fractionalSeconds)) // time doubled in JVM zone + .addRoundTrip(format("datetime64(%d)", precision), format("'2018-10-28 03:33:33%s'", fractionalSeconds), createTimestampType(precision), format("TIMESTAMP '2018-10-28 03:33:33%s'", fractionalSeconds)); // time double in Vilnius + }); + datetime64Tests.execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.test_datetime64")); + + // ClickHouse Timestamp and DateTime do not have sub-second precision timestampTest("timestamp") .execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.test_timestamp")); timestampTest("datetime") @@ -998,11 +1023,37 @@ private void testClickHouseDateTimeMinMaxValues(String timestamp) } } + @Test + public void testClickHouseDateTime64MinMaxValues() + { + testClickHouseDateTime64MinMaxValues(0, "1900-01-01 00:00:00"); + testClickHouseDateTime64MinMaxValues(9, "2262-04-11 23:47:15.999999999"); + } + + private void testClickHouseDateTime64MinMaxValues(int precision, String timestamp) + { + SqlDataTypeTest dateTests1 = SqlDataTypeTest.create() + .addRoundTrip(format("timestamp(%d)", precision), format("timestamp '%s'", timestamp), createTimestampType(precision), format("TIMESTAMP '%s'", timestamp)); + SqlDataTypeTest dateTests2 = SqlDataTypeTest.create() + .addRoundTrip(format("datetime64(%d)", precision), format("'%s'", timestamp), createTimestampType(precision), format("TIMESTAMP '%s'", timestamp)); + + for (ZoneId timeZoneId : timezones()) { + Session session = Session.builder(getSession()) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(timeZoneId.getId())) + .build(); + dateTests1.execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_datetime64")) + .execute(getQueryRunner(), session, trinoCreateAsSelect("test_datetime64")) + .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_datetime64")) + .execute(getQueryRunner(), session, trinoCreateAndInsert("test_datetime64")); + dateTests2.execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.test_datetime64")); + } + } + @Test public void testUnsupportedTimestamp() { - testUnsupportedTimestamp("1969-12-31 23:59:59"); // MIN_SUPPORTED_DATETIME_VALUE - 1 second - testUnsupportedTimestamp("2106-02-07 06:28:16"); // MAX_SUPPORTED_DATETIME_VALUE + 1 second + testUnsupportedTimestamp("1899-12-31 23:59:59"); // MIN_SUPPORTED_DATETIME_VALUE - 1 second + testUnsupportedTimestamp("2262-04-11 23:47:17"); // MAX_SUPPORTED_DATETIME_VALUE + 1 second } public void testUnsupportedTimestamp(String unsupportedTimestamp) @@ -1010,7 +1061,7 @@ public void testUnsupportedTimestamp(String unsupportedTimestamp) try (TestTable table = newTrinoTable("test_unsupported_timestamp", "(dt timestamp(0))")) { assertQueryFails( format("INSERT INTO %s VALUES (TIMESTAMP '%s')", table.getName(), unsupportedTimestamp), - format("Timestamp must be between %s and %s in ClickHouse: %s", MIN_SUPPORTED_DATETIME_VALUE, MAX_SUPPORTED_DATETIME_VALUE, unsupportedTimestamp)); + format("Timestamp must be between %s and %s in ClickHouse: %s", MIN_SUPPORTED_DATETIME64_VALUE, MAX_SUPPORTED_DATETIME64_VALUE, unsupportedTimestamp)); } try (TestTable table = new TestTable(onRemoteDatabase(), "tpch.test_unsupported_timestamp", "(dt datetime) ENGINE=Log")) { @@ -1056,6 +1107,8 @@ public void testClickHouseDateTimeWithTimeZone() dateTimeWithTimeZoneTest(DATETIME_TYPE_FACTORY) .execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.datetime_tz")); + dateTime64WithTimeZoneTest(DATETIME64_TYPE_FACTORY) + .execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.datetime64_tz")); } } @@ -1104,6 +1157,64 @@ private SqlDataTypeTest dateTimeWithTimeZoneTest(Function inputT return tests; } + private SqlDataTypeTest dateTime64WithTimeZoneTest(BiFunction inputTypeFactory) + { + ZoneId utc = ZoneId.of("UTC"); + SqlDataTypeTest tests = SqlDataTypeTest.create() + .addRoundTrip(format("Nullable(%s)", inputTypeFactory.apply(0, utc)), "NULL", TIMESTAMP_TZ_SECONDS, "CAST(NULL AS TIMESTAMP(0) WITH TIME ZONE)") + + // before epoch + .addRoundTrip(inputTypeFactory.apply(0, utc), "'1958-01-01 13:18:03'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1958-01-01 13:18:03 Z'") + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'1958-01-01 13:18:03'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1958-01-01 13:18:03 +05:30'") + + // epoch + .addRoundTrip(inputTypeFactory.apply(0, utc), "0", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:00:00 Z'") + .addRoundTrip(inputTypeFactory.apply(0, utc), "'1970-01-01 00:00:00'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:00:00 Z'") + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'1970-01-01 00:00:00'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:00:00 +05:30'") + + // after epoch + .addRoundTrip(inputTypeFactory.apply(0, utc), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 Z'") + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 +05:45'") + .addRoundTrip(inputTypeFactory.apply(0, ZoneId.of("GMT")), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 Z'") + .addRoundTrip(inputTypeFactory.apply(0, ZoneId.of("UTC+00:00")), "'2019-03-18 10:01:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2019-03-18 10:01:17 Z'") + + // time doubled in JVM zone + .addRoundTrip(inputTypeFactory.apply(0, utc), "'2018-10-28 01:33:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 01:33:17 Z'") + .addRoundTrip(inputTypeFactory.apply(0, JVM_ZONE), "'2018-10-28 01:33:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 01:33:17 -05:00'") + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'2018-10-28 01:33:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 01:33:17 +05:45'") + + // time doubled in Vilnius + .addRoundTrip(inputTypeFactory.apply(0, utc), "'2018-10-28 03:33:33'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 03:33:33 Z'") + .addRoundTrip(inputTypeFactory.apply(0, VILNIUS), "'2018-10-28 03:33:33'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 03:33:33 +03:00'") + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'2018-10-28 03:33:33'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 03:33:33 +05:45'") + + // time gap in JVM zone + .addRoundTrip(inputTypeFactory.apply(0, utc), "'1970-01-01 00:13:42'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:13:42 Z'") + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'1970-01-01 00:13:42'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:13:42 +05:30'") + .addRoundTrip(inputTypeFactory.apply(0, utc), "'2018-04-01 02:13:55'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-04-01 02:13:55 Z'") + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'2018-04-01 02:13:55'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-04-01 02:13:55 +05:45'") + + // time gap in Vilnius + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'2018-03-25 03:17:17'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-03-25 03:17:17 +05:45'") + + // time gap in KATHMANDU + .addRoundTrip(inputTypeFactory.apply(0, VILNIUS), "'1986-01-01 00:13:07'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1986-01-01 00:13:07 +03:00'") + + // test arbitrary time for all supported precisions + .addRoundTrip(inputTypeFactory.apply(0, KATHMANDU), "'2012-01-02 03:04:05'", createTimestampWithTimeZoneType(0), "TIMESTAMP '2012-01-02 03:04:05 +05:45'") + .addRoundTrip(inputTypeFactory.apply(1, KATHMANDU), "'2012-01-02 03:04:05.1'", createTimestampWithTimeZoneType(1), "TIMESTAMP '2012-01-02 03:04:05.1 +05:45'") + .addRoundTrip(inputTypeFactory.apply(2, KATHMANDU), "'2012-01-02 03:04:05.12'", createTimestampWithTimeZoneType(2), "TIMESTAMP '2012-01-02 03:04:05.12 +05:45'") + .addRoundTrip(inputTypeFactory.apply(3, KATHMANDU), "'2012-01-02 03:04:05.123'", createTimestampWithTimeZoneType(3), "TIMESTAMP '2012-01-02 03:04:05.123 +05:45'") + .addRoundTrip(inputTypeFactory.apply(4, KATHMANDU), "'2012-01-02 03:04:05.1234'", createTimestampWithTimeZoneType(4), "TIMESTAMP '2012-01-02 03:04:05.1234 +05:45'") + .addRoundTrip(inputTypeFactory.apply(5, KATHMANDU), "'2012-01-02 03:04:05.12345'", createTimestampWithTimeZoneType(5), "TIMESTAMP '2012-01-02 03:04:05.12345 +05:45'") + .addRoundTrip(inputTypeFactory.apply(6, KATHMANDU), "'2012-01-02 03:04:05.123456'", createTimestampWithTimeZoneType(6), "TIMESTAMP '2012-01-02 03:04:05.123456 +05:45'") + .addRoundTrip(inputTypeFactory.apply(7, KATHMANDU), "'2012-01-02 03:04:05.1234567'", createTimestampWithTimeZoneType(7), "TIMESTAMP '2012-01-02 03:04:05.1234567 +05:45'") + .addRoundTrip(inputTypeFactory.apply(8, KATHMANDU), "'2012-01-02 03:04:05.12345678'", createTimestampWithTimeZoneType(8), "TIMESTAMP '2012-01-02 03:04:05.12345678 +05:45'") + .addRoundTrip(inputTypeFactory.apply(9, KATHMANDU), "'2012-01-02 03:04:05.123456789'", createTimestampWithTimeZoneType(9), "TIMESTAMP '2012-01-02 03:04:05.123456789 +05:45'"); + + return tests; + } + private List timezones() { return ImmutableList.of( 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 99aa2ff7e1e3..3d0f13674f38 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 @@ -599,8 +599,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());