From 5ae0bf5a2033d08e956e9d024385ab40e1046062 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 TIMEZONE --- .../plugin/clickhouse/ClickHouseClient.java | 114 +++++++- ...Comparison.java => RewriteComparison.java} | 41 +-- .../clickhouse/expression/RewriteLike.java | 5 +- .../expression/RewriteStringIn.java | 5 +- .../expression/RewriteTimestampConstant.java | 52 ++++ .../clickhouse/BaseClickHouseTypeMapping.java | 261 +++++++++++++++--- 6 files changed, 413 insertions(+), 65 deletions(-) rename plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/{RewriteStringComparison.java => RewriteComparison.java} (74%) create mode 100644 plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteTimestampConstant.java 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 0bb07236f2d4fa..f8c2357de7db40 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 @@ -30,9 +30,10 @@ import io.trino.plugin.base.expression.ConnectorExpressionRewriter; import io.trino.plugin.base.expression.ConnectorExpressionRule.RewriteContext; import io.trino.plugin.base.mapping.IdentifierMapping; +import io.trino.plugin.clickhouse.expression.RewriteComparison; import io.trino.plugin.clickhouse.expression.RewriteLike; -import io.trino.plugin.clickhouse.expression.RewriteStringComparison; import io.trino.plugin.clickhouse.expression.RewriteStringIn; +import io.trino.plugin.clickhouse.expression.RewriteTimestampConstant; import io.trino.plugin.jdbc.BaseJdbcClient; import io.trino.plugin.jdbc.BaseJdbcConfig; import io.trino.plugin.jdbc.ColumnMapping; @@ -44,6 +45,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 +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.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; @@ -103,12 +109,16 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.OptionalLong; +import java.util.Set; import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; import java.util.stream.Collectors; +import static com.clickhouse.data.ClickHouseDataType.DateTime; +import static com.clickhouse.data.ClickHouseDataType.DateTime64; +import static com.clickhouse.data.ClickHouseDataType.FixedString; import static com.clickhouse.data.ClickHouseValues.convertToQuotedString; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.emptyToNull; @@ -145,12 +155,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 +185,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 +215,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); @@ -229,7 +246,9 @@ public ClickHouseClient( JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) - .add(new RewriteStringComparison()) + .add(new RewriteTimestampConstant()) + .add(new RewriteComparison(ImmutableList.of(CharType.class, VarcharType.class), ImmutableSet.of(FixedString, ClickHouseDataType.String))) + .add(new RewriteComparison(ImmutableList.of(TimestampType.class, TimestampWithTimeZoneType.class), ImmutableSet.of(DateTime, DateTime64))) .add(new RewriteStringIn()) .add(new RewriteLike()) .map("$not(value: boolean)").to("NOT value") @@ -720,9 +739,9 @@ 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()))); + } case Types.TIMESTAMP_WITH_TIMEZONE: if (columnDataType == ClickHouseDataType.DateTime) { // ClickHouse DateTime does not have sub-second precision @@ -732,6 +751,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) { @@ -787,12 +809,85 @@ 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 timestampWithTimeZoneType) { + return timestampWithTimeZoneWriteMapping(timestampWithTimeZoneType); + } if (type.equals(uuidType)) { return WriteMapping.sliceMapping("UUID", uuidWriteFunction()); } throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type); } + private WriteMapping timestampWriteMapping(TimestampType timestampType) + { + int precision = timestampType.getPrecision(); + String dataType = "DateTime64(%s)".formatted(precision); + if (precision <= TimestampType.MAX_SHORT_PRECISION) { + 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(timestampType, precision)); + } + + private static ColumnMapping timestampWithTimeZoneColumnMapping(ClickHouseColumn clickHouseColumn) + { + int precision = clickHouseColumn.getScale(); + TimeZone columnTimeZone = clickHouseColumn.getTimeZone(); + checkArgument(precision <= CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "unsupported precision value %s", precision); + TimestampWithTimeZoneType trinoType = createTimestampWithTimeZoneType(precision); + if (precision <= TimestampWithTimeZoneType.MAX_SHORT_PRECISION) { + return ColumnMapping.longMapping( + trinoType, + shortTimestampWithTimeZoneReadFunction(), + shortTimestampWithTimeZoneWriteFunction(columnTimeZone)); + } + return ColumnMapping.objectMapping( + trinoType, + longTimestampWithTimeZoneReadFunction(), + longTimestampWithTimeZoneWriteFunction(columnTimeZone)); + } + + private static WriteMapping timestampWithTimeZoneWriteMapping(TimestampWithTimeZoneType timestampWithTimeZoneType) + { + verify(timestampWithTimeZoneType.getPrecision() <= CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + // Clickhouse DateTime64(precision, [timezone]) + // 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 this point of time + // So create columns within UTC timezone + // TODO get timezone from session? + String dataType = format("DateTime64(%d, 'UTC')", timestampWithTimeZoneType.getPrecision()); + if (timestampWithTimeZoneType.isShort()) { + return WriteMapping.longMapping(dataType, shortTimestampWithTimeZoneWriteFunction(TimeZone.getTimeZone("UTC"))); + } + return WriteMapping.objectMapping(dataType, longTimestampWithTimeZoneWriteFunction(TimeZone.getTimeZone("UTC"))); + } + + 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 adjustNanoSeconds = (long) Math.floorMod(epochMillis, MILLISECONDS_PER_SECOND) * NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND; + Instant instant = Instant.ofEpochSecond(epochSeconds, adjustNanoSeconds); + statement.setObject(index, ZonedDateTime.ofInstant(instant, columnTimeZone.toZoneId())); + }); + } + + private ClickHouseVersion getClickHouseServerVersion(ConnectorSession session) { return clickHouseVersion.updateAndGet(current -> { @@ -984,7 +1079,7 @@ private static SliceWriteFunction uuidWriteFunction() return (statement, index, value) -> statement.setObject(index, trinoUuidToJavaUuid(value), Types.OTHER); } - public static boolean supportsPushdown(Variable variable, RewriteContext context) + public static boolean supportsPushdown(Variable variable, RewriteContext context, Set nativeTypes) { JdbcTypeHandle typeHandle = ((JdbcColumnHandle) context.getAssignment(variable.getName())) .getJdbcTypeHandle(); @@ -992,9 +1087,6 @@ public static boolean supportsPushdown(Variable variable, RewriteContext new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); ClickHouseColumn column = ClickHouseColumn.of("", jdbcTypeName); ClickHouseDataType columnDataType = column.getDataType(); - return switch (columnDataType) { - case FixedString, String -> true; - default -> false; - }; + return nativeTypes.contains(columnDataType); } } diff --git a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteStringComparison.java b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteComparison.java similarity index 74% rename from plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteStringComparison.java rename to plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteComparison.java index c6b6b1709ca7fd..8153e0dbfcfed2 100644 --- a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteStringComparison.java +++ b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteComparison.java @@ -13,7 +13,9 @@ */ package io.trino.plugin.clickhouse.expression; +import com.clickhouse.data.ClickHouseDataType; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import io.trino.matching.Capture; import io.trino.matching.Captures; import io.trino.matching.Pattern; @@ -24,9 +26,8 @@ import io.trino.spi.expression.Call; import io.trino.spi.expression.ConnectorExpression; import io.trino.spi.expression.Variable; -import io.trino.spi.type.CharType; -import io.trino.spi.type.VarcharType; +import java.util.List; import java.util.Optional; import java.util.stream.Stream; @@ -43,26 +44,34 @@ import static io.trino.spi.type.BooleanType.BOOLEAN; import static java.lang.String.format; -public class RewriteStringComparison +public class RewriteComparison implements ConnectorExpressionRule { private static final Capture LEFT = newCapture(); private static final Capture RIGHT = newCapture(); - private static final Pattern PATTERN = call() - .with(type().equalTo(BOOLEAN)) - .with(functionName().matching(Stream.of(ComparisonOperator.values()) - .filter(comparison -> comparison != ComparisonOperator.IDENTICAL) - .map(ComparisonOperator::getFunctionName) - .collect(toImmutableSet()) - ::contains)) - .with(argumentCount().equalTo(2)) - .with(argument(0).matching(expression().with(type().matching(type -> type instanceof CharType || type instanceof VarcharType)).capturedAs(LEFT))) - .with(argument(1).matching(expression().with(type().matching(type -> type instanceof CharType || type instanceof VarcharType)).capturedAs(RIGHT))); + + private final Pattern pattern; + private final ImmutableSet nativeTypes; + + public RewriteComparison(List> classes, ImmutableSet nativeTypes) + { + pattern = call() + .with(type().equalTo(BOOLEAN)) + .with(functionName().matching(Stream.of(ComparisonOperator.values()) + .filter(comparison -> comparison != ComparisonOperator.IDENTICAL) + .map(ComparisonOperator::getFunctionName) + .collect(toImmutableSet()) + ::contains)) + .with(argumentCount().equalTo(2)) + .with(argument(0).matching(expression().with(type().matching(type -> classes.stream().anyMatch(aClass -> aClass.isInstance(type)))).capturedAs(LEFT))) + .with(argument(1).matching(expression().with(type().matching(type -> classes.stream().anyMatch(aClass -> aClass.isInstance(type)))).capturedAs(RIGHT))); + this.nativeTypes = nativeTypes; + } @Override public Pattern getPattern() { - return PATTERN; + return pattern; } @Override @@ -72,11 +81,11 @@ public Optional rewrite(Call expression, Captures captu ConnectorExpression leftExpression = captures.get(LEFT); ConnectorExpression rightExpression = captures.get(RIGHT); - if (leftExpression instanceof Variable variable && !supportsPushdown(variable, context)) { + if (leftExpression instanceof Variable variable && !supportsPushdown(variable, context, nativeTypes)) { return Optional.empty(); } - if (rightExpression instanceof Variable variable && !supportsPushdown(variable, context)) { + if (rightExpression instanceof Variable variable && !supportsPushdown(variable, context, nativeTypes)) { return Optional.empty(); } diff --git a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteLike.java b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteLike.java index d3f2aa5efa7c99..8917b28024f2b3 100644 --- a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteLike.java +++ b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteLike.java @@ -13,7 +13,9 @@ */ package io.trino.plugin.clickhouse.expression; +import com.clickhouse.data.ClickHouseDataType; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import io.airlift.slice.Slice; import io.trino.matching.Capture; import io.trino.matching.Captures; @@ -29,6 +31,7 @@ import java.util.Optional; +import static com.clickhouse.data.ClickHouseDataType.FixedString; import static com.google.common.base.Preconditions.checkArgument; import static io.trino.matching.Capture.newCapture; import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.argument; @@ -59,7 +62,7 @@ public class RewriteLike .with(argumentCount().equalTo(2)) .with(argument(0).matching(variable() .with(type().matching(type -> type instanceof CharType || type instanceof VarcharType)) - .matching((Variable variable, RewriteContext context) -> supportsPushdown(variable, context)) + .matching((Variable variable, RewriteContext context) -> supportsPushdown(variable, context, ImmutableSet.of(FixedString, ClickHouseDataType.String))) .capturedAs(LIKE_VALUE))) .with(argument(1).matching(constant() .with(type().matching(type -> type instanceof CharType || type instanceof VarcharType)) diff --git a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteStringIn.java b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteStringIn.java index 546af6a32fd850..2c5925845058c1 100644 --- a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteStringIn.java +++ b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteStringIn.java @@ -13,8 +13,10 @@ */ package io.trino.plugin.clickhouse.expression; +import com.clickhouse.data.ClickHouseDataType; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import io.trino.matching.Capture; import io.trino.matching.Captures; import io.trino.matching.Pattern; @@ -31,6 +33,7 @@ import java.util.List; import java.util.Optional; +import static com.clickhouse.data.ClickHouseDataType.FixedString; import static com.google.common.base.Verify.verify; import static io.trino.matching.Capture.newCapture; import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.argument; @@ -59,7 +62,7 @@ public class RewriteStringIn .with(argumentCount().equalTo(2)) .with(argument(0).matching(variable() .with(type().matching(type -> type instanceof CharType || type instanceof VarcharType)) - .matching((Variable variable, RewriteContext context) -> supportsPushdown(variable, context)) + .matching((Variable variable, RewriteContext context) -> supportsPushdown(variable, context, ImmutableSet.of(FixedString, ClickHouseDataType.String))) .capturedAs(VALUE))) .with(argument(1).matching(call() .with(functionName().equalTo(ARRAY_CONSTRUCTOR_FUNCTION_NAME)) diff --git a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteTimestampConstant.java b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteTimestampConstant.java new file mode 100644 index 00000000000000..aebd4f1fd39837 --- /dev/null +++ b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/expression/RewriteTimestampConstant.java @@ -0,0 +1,52 @@ +/* + * 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.expression; + +import com.google.common.collect.ImmutableList; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.base.expression.ConnectorExpressionRule; +import io.trino.plugin.jdbc.QueryParameter; +import io.trino.plugin.jdbc.expression.ParameterizedExpression; +import io.trino.spi.expression.Constant; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TimestampWithTimeZoneType; + +import java.util.Optional; + +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.constant; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.type; + +public class RewriteTimestampConstant + implements ConnectorExpressionRule +{ + private static final Pattern PATTERN = constant().with(type().matching(type -> type instanceof TimestampType || type instanceof TimestampWithTimeZoneType)); + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + @Override + public Optional rewrite(Constant constant, Captures captures, RewriteContext context) + { + Object value = constant.getValue(); + if (value == null) { + // TODO we could handle NULL values too + return Optional.empty(); + } + return Optional.of(new ParameterizedExpression("?", ImmutableList.of(new QueryParameter(constant.getType(), Optional.of(value))))); + } +} 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 2444ea4bf6c6ee..3340fba81ec4cd 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,7 +16,9 @@ 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.sql.planner.plan.FilterNode; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.TestingSession; import io.trino.testing.datatype.CreateAndInsertDataSetup; @@ -36,10 +38,13 @@ 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.clickhouse.ClickHouseSessionProperties.MAP_STRING_AS_VARCHAR; import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.UNSUPPORTED_TYPE_HANDLING; import static io.trino.plugin.jdbc.UnsupportedTypeHandling.CONVERT_TO_VARCHAR; import static io.trino.spi.type.BigintType.BIGINT; @@ -51,6 +56,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 +65,7 @@ import static io.trino.type.IpAddressType.IPADDRESS; import static java.lang.String.format; import static java.time.ZoneOffset.UTC; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -914,6 +921,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 +1019,229 @@ 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 clickhouseCreateAndInsertTrinoTests = SqlDataTypeTest.create() + .addRoundTrip("Nullable(DateTime('UTC'))", "NULL", TIMESTAMP_TZ_SECONDS, "CAST(NULL AS TIMESTAMP(0) WITH TIME ZONE)"); + addClickhouseCreateAndTrinoInsertTestCases(clickhouseCreateAndInsertTrinoTests, "DateTime('%s')"::formatted, 0); + IntStream.rangeClosed(0, CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION).forEach(precision -> + addClickhouseCreateAndTrinoInsertTestCases(clickhouseCreateAndInsertTrinoTests, zone -> "DateTime64(%s, '%s')".formatted(precision, zone), precision)); + clickhouseCreateAndInsertTrinoTests.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)"); + addClickhouseCreateAndInsertTestCases(clickhouseCreateAndInsertTests, "DateTime('%s')"::formatted, 0); + IntStream.rangeClosed(0, CLICKHOUSE_MAX_SUPPORTED_TIMESTAMP_PRECISION).forEach(precision -> + addClickhouseCreateAndInsertTestCases(clickhouseCreateAndInsertTests, zone -> "DateTime64(%s, '%s')".formatted(precision, zone), precision)); + clickhouseCreateAndInsertTests.execute(getQueryRunner(), session, clickhouseCreateAndInsert("tpch.datetime_tz")); } } - private SqlDataTypeTest dateTimeWithTimeZoneTest(Function inputTypeFactory) + private void addClickhouseCreateAndTrinoInsertTestCases(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 addClickhouseCreateAndInsertTestCases(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'"); + .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)); + } - return tests; + @Test + public void testDateTimePredicatePushdown() + { + Session mapStringAsVarchar = Session.builder(getSession()) + .setCatalogSessionProperty("clickhouse", MAP_STRING_AS_VARCHAR, Boolean.toString(true)) + .build(); + String withConnectorExpression = " OR some_column = 'x'"; + try (TestTable table = new TestTable( + onRemoteDatabase(), + "tpch.test_datetime_predicate_pushdown", + """ + ( + some_column String, + c_datetime_1 DateTime, + c_datetime_2 DateTime, + c_datetime64_3_1 DateTime64, + c_datetime64_3_2 DateTime64, + c_datetime64_9_1 DateTime64(9), + c_datetime64_9_2 DateTime64(9), + c_datetime64_3_zone_1 DateTime64(3, 'Asia/Istanbul'), + c_datetime64_3_zone_2 DateTime64(3, 'Asia/Istanbul'), + c_datetime64_9_zone_1 DateTime64(9, 'Asia/Istanbul'), + c_datetime64_9_zone_2 DateTime64(9, 'Asia/Istanbul') + ) + ENGINE=Log""", + List.of( + "'1', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.000', '2024-01-01 00:00:00.000', '2024-01-01 00:00:00.000', '2024-01-01 00:00:00.000', '2024-01-01 00:00:00.000', '2024-01-01 00:00:00.000', '2024-01-01 00:00:00.000', '2024-01-01 00:00:00.000'", + "'2', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.123'", + "'3', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.1234'", + "'4', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1234', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1234'", + "'5', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1239', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1239', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1239', '2024-01-01 00:00:00.123', '2024-01-01 00:00:00.1239'", + "'6', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.1238', '2024-01-01 00:00:00.1239', '2024-01-01 00:00:00.1238', '2024-01-01 00:00:00.1239', '2024-01-01 00:00:00.1238', '2024-01-01 00:00:00.1239', '2024-01-01 00:00:00.1238', '2024-01-01 00:00:00.1239'", + "'7', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456787', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456787', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456787', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456787'", + "'8', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456788', '2024-01-01 00:00:00.123456789'", + "'9', '2024-01-01 00:00:00', '2024-01-01 00:00:00', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456789', '2024-01-01 00:00:00.123456789'" + ))) { + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime_1 = TIMESTAMP '2024-01-01 00:00:00'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime_1 = TIMESTAMP '2024-01-01 00:00:00'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.0'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.0'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.00'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.00'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.000'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.000'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.0000'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 00:00:00.0000'" + withConnectorExpression)).isFullyPushedDown(); + + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime_1 = TIMESTAMP '2024-01-01 12:34:56'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime_1 = TIMESTAMP '2024-01-01 12:34:56'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.1'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.1'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.12'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.12'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.123'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.123'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.1234'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.1234'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.12345'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.12345'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.123456'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.123456'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = c_datetime64_3_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 = c_datetime64_3_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 < c_datetime64_3_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 < c_datetime64_3_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 > c_datetime64_3_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_1 > c_datetime64_3_2" + withConnectorExpression)).isFullyPushedDown(); + + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.1'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.1'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.12'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.12'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.123'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.123'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.1234'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.1234'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.12345'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.12345'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.123456'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.123456'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = c_datetime64_9_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 = c_datetime64_9_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 < c_datetime64_9_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 < c_datetime64_9_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 > c_datetime64_9_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_1 > c_datetime64_9_2" + withConnectorExpression)).isFullyPushedDown(); + + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'" + withConnectorExpression)).isNotFullyPushedDown(FilterNode.class); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = c_datetime64_3_zone_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 = c_datetime64_3_zone_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 < c_datetime64_3_zone_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 < c_datetime64_3_zone_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 > c_datetime64_3_zone_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_3_zone_1 > c_datetime64_3_zone_2" + withConnectorExpression)).isFullyPushedDown(); + + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123'")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.1234567'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.12345678'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'")).returnsEmptyResult(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = TIMESTAMP '2024-01-01 12:34:56.123456789'" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = c_datetime64_9_zone_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 = c_datetime64_9_zone_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 < c_datetime64_9_zone_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 < c_datetime64_9_zone_2" + withConnectorExpression)).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 > c_datetime64_9_zone_2")).isFullyPushedDown(); + assertThat(query(mapStringAsVarchar, "SELECT some_column FROM " + table.getName() + " WHERE c_datetime64_9_zone_1 > c_datetime64_9_zone_2" + withConnectorExpression)).isFullyPushedDown(); + } } private List timezones() @@ -1177,8 +1360,14 @@ protected SqlExecutor onRemoteDatabase() return clickhouseServer::execute; } - private static Function clickhouseDateTimeInputTypeFactory(String inputTypePrefix) - { - return zone -> format("%s('%s')", inputTypePrefix, zone); - } +// @RepeatedTest(1000) +// @Execution(SAME_THREAD) +// public void testRepeated() +// { +// testTimestamp(); +// testClickHouseDateTimeMinMaxValues(); +// testUnsupportedTimestamp(); +// testClickHouseDateTimeWithTimeZone(); +// testDateTimePredicatePushdown(); +// } }