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 c2846855257d..515ff2608117 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 @@ -636,6 +636,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect return mapping; } + ClickHouseVersion version = getClickHouseServerVersion(session); ClickHouseColumn column = ClickHouseColumn.of("", jdbcTypeName); ClickHouseDataType columnDataType = column.getDataType(); switch (columnDataType) { @@ -724,7 +725,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect DISABLE_PUSHDOWN)); case Types.DATE: - return Optional.of(dateColumnMappingUsingLocalDate(getClickHouseServerVersion(session))); + return Optional.of(dateColumnMappingUsingLocalDate(version)); case Types.TIMESTAMP: if (columnDataType == ClickHouseDataType.DateTime) { @@ -733,7 +734,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect return Optional.of(ColumnMapping.longMapping( TIMESTAMP_SECONDS, timestampReadFunction(TIMESTAMP_SECONDS), - timestampSecondsWriteFunction(getClickHouseServerVersion(session)))); + timestampSecondsWriteFunction(version))); } // TODO (https://github.com/trinodb/trino/issues/10537) Add support for Datetime64 type return Optional.of(timestampColumnMapping(TIMESTAMP_MILLIS)); @@ -745,7 +746,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect return Optional.of(ColumnMapping.longMapping( TIMESTAMP_TZ_SECONDS, shortTimestampWithTimeZoneReadFunction(), - shortTimestampWithTimeZoneWriteFunction(column.getTimeZone()))); + shortTimestampWithTimeZoneWriteFunction(version, column.getTimeZone()))); } } @@ -926,12 +927,15 @@ private static LongReadFunction shortTimestampWithTimeZoneReadFunction() }; } - private static LongWriteFunction shortTimestampWithTimeZoneWriteFunction(TimeZone columnTimeZone) + private static LongWriteFunction shortTimestampWithTimeZoneWriteFunction(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. - statement.setObject(index, Instant.ofEpochMilli(millisUtc).atZone(columnTimeZone.toZoneId())); + Instant instant = Instant.ofEpochMilli(millisUtc); + // ClickHouse stores incorrect results when the values are out of supported range. + DATETIME.validate(version, instant.atZone(UTC).toLocalDateTime()); + statement.setObject(index, instant.atZone(columnTimeZone.toZoneId())); }; } 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 a37c61dd7404..d42a6d9024b4 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 @@ -59,37 +59,40 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static io.trino.type.IpAddressType.IPADDRESS; import static java.lang.String.format; -import static java.time.ZoneOffset.UTC; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) public abstract class BaseClickHouseTypeMapping extends AbstractTestQueryFramework { - private final ZoneId jvmZone = ZoneId.systemDefault(); - + private static final ZoneId UTC = ZoneId.of("UTC"); + private static final ZoneId JVM_ZONE = ZoneId.systemDefault(); // no DST in 1970, but has DST in later years (e.g. 2018) - private final ZoneId vilnius = ZoneId.of("Europe/Vilnius"); - + private static final ZoneId VILNIUS = ZoneId.of("Europe/Vilnius"); // minutes offset change since 1932-04-01, no DST - private final ZoneId kathmandu = ZoneId.of("Asia/Kathmandu"); + private static final ZoneId KATHMANDU = ZoneId.of("Asia/Kathmandu"); + private static final Function DATETIME_TYPE_FACTORY = "DateTime('%s')"::formatted; + + // 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"; protected TestingClickHouseServer clickhouseServer; @BeforeAll public void setUp() { - checkState(jvmZone.getId().equals("America/Bahia_Banderas"), "This test assumes certain JVM time zone"); + checkState(JVM_ZONE.getId().equals("America/Bahia_Banderas"), "This test assumes certain JVM time zone"); LocalDate dateOfLocalTimeChangeForwardAtMidnightInJvmZone = LocalDate.of(1932, 4, 1); - checkIsGap(jvmZone, dateOfLocalTimeChangeForwardAtMidnightInJvmZone.atStartOfDay()); + checkIsGap(JVM_ZONE, dateOfLocalTimeChangeForwardAtMidnightInJvmZone.atStartOfDay()); LocalDate dateOfLocalTimeChangeForwardAtMidnightInSomeZone = LocalDate.of(1983, 4, 1); - checkIsGap(vilnius, dateOfLocalTimeChangeForwardAtMidnightInSomeZone.atStartOfDay()); + checkIsGap(VILNIUS, dateOfLocalTimeChangeForwardAtMidnightInSomeZone.atStartOfDay()); LocalDate dateOfLocalTimeChangeBackwardAtMidnightInSomeZone = LocalDate.of(1983, 10, 1); - checkIsDoubled(vilnius, dateOfLocalTimeChangeBackwardAtMidnightInSomeZone.atStartOfDay().minusMinutes(1)); + checkIsDoubled(VILNIUS, dateOfLocalTimeChangeBackwardAtMidnightInSomeZone.atStartOfDay().minusMinutes(1)); LocalDate timeGapInKathmandu = LocalDate.of(1986, 1, 1); - checkIsGap(kathmandu, timeGapInKathmandu.atStartOfDay()); + checkIsGap(KATHMANDU, timeGapInKathmandu.atStartOfDay()); } private static void checkIsGap(ZoneId zone, LocalDateTime dateTime) @@ -968,8 +971,8 @@ protected SqlDataTypeTest unsupportedTimestampBecomeUnexpectedValueTest(String i @Test public void testClickHouseDateTimeMinMaxValues() { - testClickHouseDateTimeMinMaxValues("1970-01-01 00:00:00"); // min value in ClickHouse - testClickHouseDateTimeMinMaxValues("2106-02-07 06:28:15"); // max value in ClickHouse + testClickHouseDateTimeMinMaxValues(MIN_SUPPORTED_DATETIME_VALUE); // min value in ClickHouse + testClickHouseDateTimeMinMaxValues(MAX_SUPPORTED_DATETIME_VALUE); // max value in ClickHouse } private void testClickHouseDateTimeMinMaxValues(String timestamp) @@ -998,19 +1001,16 @@ private void testClickHouseDateTimeMinMaxValues(String timestamp) @Test public void testUnsupportedTimestamp() { - testUnsupportedTimestamp("1969-12-31 23:59:59"); // min - 1 second - testUnsupportedTimestamp("2106-02-07 06:28:16"); // max + 1 second + 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 } public void testUnsupportedTimestamp(String unsupportedTimestamp) { - String minSupportedTimestamp = "1970-01-01 00:00:00"; - String maxSupportedTimestamp = "2106-02-07 06:28:15"; - 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", minSupportedTimestamp, maxSupportedTimestamp, unsupportedTimestamp)); + format("Timestamp must be between %s and %s in ClickHouse: %s", MIN_SUPPORTED_DATETIME_VALUE, MAX_SUPPORTED_DATETIME_VALUE, unsupportedTimestamp)); } try (TestTable table = new TestTable(onRemoteDatabase(), "tpch.test_unsupported_timestamp", "(dt datetime) ENGINE=Log")) { @@ -1019,6 +1019,27 @@ public void testUnsupportedTimestamp(String unsupportedTimestamp) } } + @Test + void testUnsupportedDateTimeWithTimeZone() + { + for (ZoneId zoneId : timezones()) { + String inputType = DATETIME_TYPE_FACTORY.apply(zoneId); + testUnsupportedDateTimeWithTimeZone(inputType, "1969-12-31 23:59:59 UTC", "1969-12-31 23:59:59"); // MIN_SUPPORTED_DATETIME_VALUE - 1 second + testUnsupportedDateTimeWithTimeZone(inputType, "2106-02-07 06:28:16 UTC", "2106-02-07 06:28:16"); // MAX_SUPPORTED_DATETIME_VALUE + 1 second + testUnsupportedDateTimeWithTimeZone(inputType, "1970-01-01 00:00:00 Asia/Kathmandu", "1969-12-31 18:30:00"); + testUnsupportedDateTimeWithTimeZone(inputType, "1970-01-01 00:13:42 Asia/Kathmandu", "1969-12-31 18:43:42"); + } + } + + private void testUnsupportedDateTimeWithTimeZone(String inputType, String unsupportedTimestampWithTz, String unsupportedTimestampUtc) + { + try (TestTable table = new TestTable(onRemoteDatabase(), "tpch.test_unsupported_timestamp_with_tz", "(dt %s) ENGINE=Log".formatted(inputType))) { + assertQueryFails( + "INSERT INTO %s VALUES (TIMESTAMP '%s')".formatted(table.getName(), unsupportedTimestampWithTz), + "Timestamp must be between %s and %s in ClickHouse: %s".formatted(MIN_SUPPORTED_DATETIME_VALUE, MAX_SUPPORTED_DATETIME_VALUE, unsupportedTimestampUtc)); + } + } + @Test public void testClickHouseDateTimeWithTimeZone() { @@ -1033,53 +1054,52 @@ public void testClickHouseDateTimeWithTimeZone() .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")) + dateTimeWithTimeZoneTest(DATETIME_TYPE_FACTORY) .execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.datetime_tz")); } } private SqlDataTypeTest dateTimeWithTimeZoneTest(Function inputTypeFactory) { - 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)") + .addRoundTrip(format("Nullable(%s)", inputTypeFactory.apply(UTC)), "NULL", TIMESTAMP_TZ_SECONDS, "CAST(NULL AS TIMESTAMP(0) WITH TIME ZONE)") // 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", 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'") + // DateTime supports the range [1970-01-01 00:00:00, 2106-02-07 06:28:15] + // Values outside this range gets stored incorrectly in ClickHouse. + // For example, 1970-01-01 00:00:00 in Asia/Kathmandu could be stored as 1970-01-01 05:30:00 + .addRoundTrip(inputTypeFactory.apply(KATHMANDU), "'1970-01-01 00:00:00'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 05:30:00 +05:30'") // 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(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'") // 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'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-10-28 01:33:17 Z'") + .addRoundTrip(inputTypeFactory.apply(JVM_ZONE), "'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'") // 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'", 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'") // 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'") - // 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), "'1970-01-01 00:13:42'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1970-01-01 00:13:42 Z'") + .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'") // 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'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '2018-03-25 03:17:17 +05:45'") // 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'"); + .addRoundTrip(inputTypeFactory.apply(VILNIUS), "'1986-01-01 00:13:07'", TIMESTAMP_TZ_SECONDS, "TIMESTAMP '1986-01-01 00:13:07 +03:00'"); return tests; } @@ -1088,10 +1108,10 @@ private List timezones() { return ImmutableList.of( UTC, - jvmZone, + JVM_ZONE, // using two non-JVM zones so that we don't need to worry what ClickHouse system zone is - vilnius, - kathmandu, + VILNIUS, + KATHMANDU, TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId()); } @@ -1199,9 +1219,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/TestAltinityClickHouseTypeMapping.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestAltinityClickHouseTypeMapping.java new file mode 100644 index 000000000000..a15fd8751090 --- /dev/null +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestAltinityClickHouseTypeMapping.java @@ -0,0 +1,40 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.clickhouse; + +import io.trino.testing.QueryRunner; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.clickhouse.TestingClickHouseServer.ALTINITY_DEFAULT_IMAGE; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +final class TestAltinityClickHouseTypeMapping + extends BaseClickHouseTypeMapping +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + clickhouseServer = closeAfterClass(new TestingClickHouseServer(ALTINITY_DEFAULT_IMAGE)); + return ClickHouseQueryRunner.builder(clickhouseServer).build(); + } + + @Override + @Test + public void testUnsupportedPoint() + { + assertThatThrownBy(super::testUnsupportedPoint) + .hasMessageContaining("Failed to execute statement: CREATE TABLE"); + } +} diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestAltinityLatestClickHouseTypeMapping.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestAltinityLatestClickHouseTypeMapping.java new file mode 100644 index 000000000000..e1ce24fc308c --- /dev/null +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestAltinityLatestClickHouseTypeMapping.java @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.clickhouse; + +import io.trino.testing.QueryRunner; + +import static io.trino.plugin.clickhouse.TestingClickHouseServer.ALTINITY_LATEST_IMAGE; + +final class TestAltinityLatestClickHouseTypeMapping + extends BaseClickHouseTypeMapping +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + clickhouseServer = closeAfterClass(new TestingClickHouseServer(ALTINITY_LATEST_IMAGE)); + return ClickHouseQueryRunner.builder(clickhouseServer).build(); + } +}