Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -636,6 +636,7 @@ public Optional<ColumnMapping> toColumnMapping(ConnectorSession session, Connect
return mapping;
}

ClickHouseVersion version = getClickHouseServerVersion(session);
ClickHouseColumn column = ClickHouseColumn.of("", jdbcTypeName);
ClickHouseDataType columnDataType = column.getDataType();
switch (columnDataType) {
Expand Down Expand Up @@ -724,7 +725,7 @@ public Optional<ColumnMapping> 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) {
Expand All @@ -733,7 +734,7 @@ public Optional<ColumnMapping> 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));
Expand All @@ -745,7 +746,7 @@ public Optional<ColumnMapping> toColumnMapping(ConnectorSession session, Connect
return Optional.of(ColumnMapping.longMapping(
TIMESTAMP_TZ_SECONDS,
shortTimestampWithTimeZoneReadFunction(),
shortTimestampWithTimeZoneWriteFunction(column.getTimeZone())));
shortTimestampWithTimeZoneWriteFunction(version, column.getTimeZone())));
}
}

Expand Down Expand Up @@ -926,12 +927,15 @@ private static LongReadFunction shortTimestampWithTimeZoneReadFunction()
};
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should validate the reading as well

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When an out-of-range value is inserted into a ClickHouse table, ClickHouse stores it incorrectly by mapping it to a value within the supported range.

Example:
1912-01-01 00:00:00 → stored as 1970-01-01 00:00:00
3102-01-01 00:00:00 → stored as 2027-10-17 11:03:28

In Trino, the value that gets read appears to be within the supported range, making it impossible to tell whether 1970-01-01 00:00:00 was originally 1912-01-01 00:00:00 or actually 1970-01-01 00:00:00.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

3102-01-01 00:00:00 → stored as 2027-10-17 11:03:28

Just curious: If the given date above MAX it's stored not as MAX (2106-02-07 06:28:15) but as 2027-10-17 11:03:28 ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I have checked with some different values as well and it is inserting different values for those cases.

}

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()));
};
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<ZoneId, String> 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)
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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")) {
Expand All @@ -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()
{
Expand All @@ -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<ZoneId, String> 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;
}
Expand All @@ -1088,10 +1108,10 @@ private List<ZoneId> 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());
}

Expand Down Expand Up @@ -1199,9 +1219,4 @@ protected SqlExecutor onRemoteDatabase()
{
return clickhouseServer::execute;
}

private static Function<ZoneId, String> clickhouseDateTimeInputTypeFactory(String inputTypePrefix)
{
return zone -> format("%s('%s')", inputTypePrefix, zone);
}
}
Original file line number Diff line number Diff line change
@@ -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");
}
}
Original file line number Diff line number Diff line change
@@ -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();
}
}