From 1fda3373df1b4ea49f420315e6c8d5c99be7ace6 Mon Sep 17 00:00:00 2001
From: "mykhailo.skliar"
Date: Mon, 4 Aug 2025 05:14:09 +0200
Subject: [PATCH] Add support for TIMESTAMP types in Exasol connector
---
docs/src/main/sphinx/connector/exasol.md | 6 +
.../io/trino/plugin/exasol/ExasolClient.java | 284 ++++++++++++++++++
.../plugin/exasol/TestExasolTypeMapping.java | 264 +++++++++++++++-
.../plugin/exasol/TestingExasolServer.java | 22 ++
4 files changed, 562 insertions(+), 14 deletions(-)
diff --git a/docs/src/main/sphinx/connector/exasol.md b/docs/src/main/sphinx/connector/exasol.md
index 5f0c29a89f4b..2f069f5ac563 100644
--- a/docs/src/main/sphinx/connector/exasol.md
+++ b/docs/src/main/sphinx/connector/exasol.md
@@ -103,6 +103,12 @@ Trino data type mapping:
* - `HASHTYPE`
- `VARBINARY`
-
+* - `TIMESTAMP(n)`
+ - `TIMESTAMP(n)`
+ -
+* - `TIMESTAMP(n) WITH LOCAL TIME ZONE`
+ - `TIMESTAMP(n) WITH TIME ZONE`
+ -
:::
No other types are supported.
diff --git a/plugin/trino-exasol/src/main/java/io/trino/plugin/exasol/ExasolClient.java b/plugin/trino-exasol/src/main/java/io/trino/plugin/exasol/ExasolClient.java
index 8675aa509435..12dde6c89396 100644
--- a/plugin/trino-exasol/src/main/java/io/trino/plugin/exasol/ExasolClient.java
+++ b/plugin/trino-exasol/src/main/java/io/trino/plugin/exasol/ExasolClient.java
@@ -30,6 +30,8 @@
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.SliceReadFunction;
import io.trino.plugin.jdbc.SliceWriteFunction;
@@ -43,34 +45,57 @@
import io.trino.spi.connector.ColumnPosition;
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.connector.ConnectorTableMetadata;
+import io.trino.spi.type.LongTimestamp;
+import io.trino.spi.type.LongTimestampWithTimeZone;
+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 java.sql.Connection;
import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Timestamp;
import java.sql.Types;
+import java.time.Instant;
import java.time.LocalDate;
+import java.time.LocalDateTime;
import java.util.HexFormat;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Set;
+import java.util.TimeZone;
import java.util.function.BiFunction;
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.plugin.jdbc.PredicatePushdownController.FULL_PUSHDOWN;
import static io.trino.plugin.jdbc.StandardColumnMappings.bigintColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.booleanColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.decimalColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.defaultCharColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.defaultVarcharColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.doubleColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.fromLongTrinoTimestamp;
+import static io.trino.plugin.jdbc.StandardColumnMappings.fromTrinoTimestamp;
import static io.trino.plugin.jdbc.StandardColumnMappings.integerColumnMapping;
import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.toLongTrinoTimestamp;
+import static io.trino.plugin.jdbc.StandardColumnMappings.toTrinoTimestamp;
import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.getUnsupportedTypeHandling;
import static io.trino.plugin.jdbc.UnsupportedTypeHandling.CONVERT_TO_VARCHAR;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.spi.connector.ConnectorMetadata.MODIFYING_ROWS_MESSAGE;
+import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone;
+import static io.trino.spi.type.DateTimeEncoding.unpackMillisUtc;
import static io.trino.spi.type.DateType.DATE;
import static io.trino.spi.type.DecimalType.createDecimalType;
+import static io.trino.spi.type.TimestampType.createTimestampType;
+import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType;
+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.VarbinaryType.VARBINARY;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;
@@ -79,11 +104,15 @@
public class ExasolClient
extends BaseJdbcClient
{
+ private static final int EXASOL_TIMESTAMP_WITH_TIMEZONE = 124;
+
private static final Set INTERNAL_SCHEMAS = ImmutableSet.builder()
.add("EXA_STATISTICS")
.add("SYS")
.build();
+ private static final int MAX_EXASOL_TIMESTAMP_PRECISION = 9;
+
@Inject
public ExasolClient(
BaseJdbcConfig config,
@@ -241,6 +270,10 @@ public Optional toColumnMapping(ConnectorSession session, Connect
return Optional.of(defaultVarcharColumnMapping(typeHandle.requiredColumnSize(), true));
case Types.DATE:
return Optional.of(dateColumnMapping());
+ case Types.TIMESTAMP:
+ return Optional.of(timestampColumnMapping(typeHandle));
+ case EXASOL_TIMESTAMP_WITH_TIMEZONE:
+ return Optional.of(timestampWithTimeZoneColumnMapping(typeHandle));
}
if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) {
@@ -256,6 +289,257 @@ private boolean isHashType(JdbcTypeHandle typeHandle)
&& typeHandle.jdbcTypeName().get().equalsIgnoreCase("HASHTYPE");
}
+ private static ColumnMapping timestampColumnMapping(JdbcTypeHandle typeHandle)
+ {
+ int timestampPrecision = typeHandle.requiredDecimalDigits();
+ TimestampType timestampType = createTimestampType(timestampPrecision);
+ if (timestampType.isShort()) {
+ return ColumnMapping.longMapping(
+ timestampType,
+ longTimestampReadFunction(timestampType),
+ longTimestampWriteFunction(timestampType),
+ FULL_PUSHDOWN);
+ }
+ return ColumnMapping.objectMapping(
+ timestampType,
+ objectTimestampReadFunction(timestampType),
+ objectTimestampWriteFunction(timestampType),
+ FULL_PUSHDOWN);
+ }
+
+ private static LongReadFunction longTimestampReadFunction(TimestampType timestampType)
+ {
+ return (resultSet, columnIndex) -> {
+ Timestamp timestamp = resultSet.getObject(columnIndex, Timestamp.class);
+ return toTrinoTimestamp(timestampType, timestamp.toLocalDateTime());
+ };
+ }
+
+ private static ObjectReadFunction objectTimestampReadFunction(TimestampType timestampType)
+ {
+ verifyObjectTimestampPrecision(timestampType);
+ return ObjectReadFunction.of(
+ LongTimestamp.class,
+ (resultSet, columnIndex) -> {
+ Timestamp timestamp = resultSet.getObject(columnIndex, Timestamp.class);
+ return toLongTrinoTimestamp(timestampType, timestamp.toLocalDateTime());
+ });
+ }
+
+ private static void verifyObjectTimestampPrecision(TimestampType timestampType)
+ {
+ int precision = timestampType.getPrecision();
+ checkArgument(precision > TimestampType.MAX_SHORT_PRECISION && precision <= MAX_EXASOL_TIMESTAMP_PRECISION,
+ "Precision is out of range: %s", precision);
+ }
+
+ private static ObjectWriteFunction objectTimestampWriteFunction(TimestampType timestampType)
+ {
+ int precision = timestampType.getPrecision();
+ verifyObjectTimestampPrecision(timestampType);
+
+ return new ObjectWriteFunction() {
+ @Override
+ public Class> getJavaType()
+ {
+ return LongTimestamp.class;
+ }
+
+ @Override
+ public void set(PreparedStatement statement, int index, Object value)
+ throws SQLException
+ {
+ LocalDateTime localDateTime = fromLongTrinoTimestamp((LongTimestamp) value, precision);
+ Timestamp timestamp = Timestamp.valueOf(localDateTime);
+ statement.setObject(index, timestamp);
+ }
+
+ @Override
+ public String getBindExpression()
+ {
+ return getTimestampBindExpression(timestampType);
+ }
+
+ @Override
+ public void setNull(PreparedStatement statement, int index)
+ throws SQLException
+ {
+ statement.setNull(index, Types.VARCHAR);
+ }
+ };
+ }
+
+ private static LongWriteFunction longTimestampWriteFunction(TimestampType timestampType)
+ {
+ return new LongWriteFunction()
+ {
+ @Override
+ public String getBindExpression()
+ {
+ return getTimestampBindExpression(timestampType);
+ }
+
+ @Override
+ public void set(PreparedStatement statement, int index, long epochMicros)
+ throws SQLException
+ {
+ LocalDateTime localDateTime = fromTrinoTimestamp(epochMicros);
+ Timestamp timestampValue = Timestamp.valueOf(localDateTime);
+ statement.setObject(index, timestampValue);
+ }
+
+ @Override
+ public void setNull(PreparedStatement statement, int index)
+ throws SQLException
+ {
+ statement.setNull(index, Types.VARCHAR);
+ }
+ };
+ }
+
+ private static ColumnMapping timestampWithTimeZoneColumnMapping(JdbcTypeHandle typeHandle)
+ {
+ int timestampPrecision = typeHandle.requiredDecimalDigits();
+ TimestampWithTimeZoneType timestampWithTimeZoneType = createTimestampWithTimeZoneType(timestampPrecision);
+
+ if (timestampWithTimeZoneType.isShort()) {
+ return ColumnMapping.longMapping(
+ timestampWithTimeZoneType,
+ longTimestampWithTimeZoneReadFunction(),
+ longTimestampWithTimeZoneWriteFunction(timestampWithTimeZoneType),
+ FULL_PUSHDOWN);
+ }
+ return ColumnMapping.objectMapping(
+ timestampWithTimeZoneType,
+ objectTimestampWithTimeZoneReadFunction(timestampWithTimeZoneType),
+ objectTimestampWithTimeZoneWriteFunction(timestampWithTimeZoneType),
+ FULL_PUSHDOWN);
+ }
+
+ private static LongReadFunction longTimestampWithTimeZoneReadFunction()
+ {
+ return (resultSet, columnIndex) -> {
+ Timestamp timestamp = resultSet.getObject(columnIndex, Timestamp.class);
+ return packDateTimeWithZone(timestamp.getTime(), TimeZone.getDefault().getID());
+ };
+ }
+
+ private static ObjectReadFunction objectTimestampWithTimeZoneReadFunction(
+ TimestampWithTimeZoneType timestampType)
+ {
+ verifyObjectTimestampWithTimeZonePrecision(timestampType);
+ return ObjectReadFunction.of(
+ LongTimestampWithTimeZone.class,
+ (resultSet, columnIndex) -> {
+ Timestamp timestamp = resultSet.getObject(columnIndex, Timestamp.class);
+
+ long millisUtc = timestamp.getTime();
+ long nanosUtc = millisUtc * NANOSECONDS_PER_MILLISECOND + timestamp.getNanos() % NANOSECONDS_PER_MILLISECOND;
+ int picosOfMilli = (int) ((nanosUtc - millisUtc * NANOSECONDS_PER_MILLISECOND) * PICOSECONDS_PER_NANOSECOND);
+
+ return LongTimestampWithTimeZone.fromEpochMillisAndFraction(
+ millisUtc,
+ picosOfMilli,
+ TimeZoneKey.getTimeZoneKey(TimeZone.getDefault().getID()));
+ });
+ }
+
+ private static void verifyObjectTimestampWithTimeZonePrecision(TimestampWithTimeZoneType timestampType)
+ {
+ int precision = timestampType.getPrecision();
+ checkArgument(precision > TimestampWithTimeZoneType.MAX_SHORT_PRECISION && precision <= MAX_EXASOL_TIMESTAMP_PRECISION,
+ "Precision is out of range: %s", precision);
+ }
+
+ private static ObjectWriteFunction objectTimestampWithTimeZoneWriteFunction(TimestampWithTimeZoneType timestampType)
+ {
+ verifyObjectTimestampWithTimeZonePrecision(timestampType);
+
+ return new ObjectWriteFunction() {
+ @Override
+ public Class> getJavaType()
+ {
+ return LongTimestampWithTimeZone.class;
+ }
+
+ @Override
+ public void set(PreparedStatement statement, int index, Object value)
+ throws SQLException
+ {
+ LongTimestampWithTimeZone longTimestampWithTimeZone = (LongTimestampWithTimeZone) value;
+ Instant instant = Instant.ofEpochMilli(longTimestampWithTimeZone.getEpochMillis())
+ .plusNanos(longTimestampWithTimeZone.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND);
+ Timestamp timestamp = Timestamp.from(instant);
+ statement.setObject(index, timestamp);
+ }
+
+ @Override
+ public String getBindExpression()
+ {
+ return getTimestampWithTimeZoneBindExpression(timestampType);
+ }
+
+ @Override
+ public void setNull(PreparedStatement statement, int index)
+ throws SQLException
+ {
+ statement.setNull(index, Types.VARCHAR);
+ }
+ };
+ }
+
+ private static LongWriteFunction longTimestampWithTimeZoneWriteFunction(TimestampWithTimeZoneType timestampType)
+ {
+ return new LongWriteFunction()
+ {
+ @Override
+ public String getBindExpression()
+ {
+ return getTimestampWithTimeZoneBindExpression(timestampType);
+ }
+
+ @Override
+ public void set(PreparedStatement statement, int index, long dateTimeWithTimeZone)
+ throws SQLException
+ {
+ long epochMillis = unpackMillisUtc(dateTimeWithTimeZone);
+ Timestamp timestampValue = Timestamp.from(Instant.ofEpochMilli(epochMillis));
+ statement.setObject(index, timestampValue);
+ }
+
+ @Override
+ public void setNull(PreparedStatement statement, int index)
+ throws SQLException
+ {
+ statement.setNull(index, Types.VARCHAR);
+ }
+ };
+ }
+
+ private static String getTimestampBindExpression(TimestampType timestampType)
+ {
+ return getTimestampBindExpression(timestampType.getPrecision());
+ }
+
+ private static String getTimestampWithTimeZoneBindExpression(TimestampWithTimeZoneType timestampWithTimeZoneType)
+ {
+ return getTimestampBindExpression(timestampWithTimeZoneType.getPrecision());
+ }
+
+ /**
+ * Returns a {@code TO_TIMESTAMP} bind expression using the appropriate format model
+ * based on the given fractional seconds precision.
+ * See for more details: https://docs.exasol.com/db/latest/sql_references/formatmodels.htm
+ */
+ private static String getTimestampBindExpression(int precision)
+ {
+ //negative precisions are not supported by TimestampType and TimestampWithTimeZoneType
+ if (precision == 0) {
+ return "TO_TIMESTAMP(?, 'YYYY-MM-DD HH24:MI:SS')";
+ }
+ return format("TO_TIMESTAMP(?, 'YYYY-MM-DD HH24:MI:SS.FF%d')", precision);
+ }
+
private static ColumnMapping dateColumnMapping()
{
// Exasol driver does not support LocalDate
diff --git a/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestExasolTypeMapping.java b/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestExasolTypeMapping.java
index ebe4a0da53da..faa3d1b6c488 100644
--- a/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestExasolTypeMapping.java
+++ b/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestExasolTypeMapping.java
@@ -40,12 +40,15 @@
import static io.trino.spi.type.DateType.DATE;
import static io.trino.spi.type.DecimalType.createDecimalType;
import static io.trino.spi.type.DoubleType.DOUBLE;
+import static io.trino.spi.type.TimestampType.createTimestampType;
+import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType;
import static io.trino.spi.type.VarbinaryType.VARBINARY;
import static io.trino.spi.type.VarcharType.createVarcharType;
import static io.trino.testing.TestingNames.randomNameSuffix;
import static java.lang.String.format;
import static java.time.ZoneOffset.UTC;
import static java.util.Arrays.asList;
+import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
@Isolated
@@ -278,6 +281,239 @@ private void testDate(ZoneId sessionZone)
.execute(getQueryRunner(), session, exasolCreateAndInsert(TEST_SCHEMA + "." + "test_date"));
}
+ // See for more details: https://docs.exasol.com/db/latest/sql_references/data_types/datatypedetails.htm
+ @Test
+ void testTimestamp()
+ {
+ testTimestamp(UTC);
+ testTimestamp(jvmZone);
+ // using two non-JVM zones so that we don't need to worry what Exasol system zone is
+ testTimestamp(vilnius);
+ testTimestamp(kathmandu);
+ testTimestamp(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId());
+ }
+
+ private void testTimestamp(ZoneId sessionZone)
+ {
+ Session session = Session.builder(getSession())
+ .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId()))
+ .build();
+
+ SqlDataTypeTest.create()
+ .addRoundTrip("timestamp", "NULL", createTimestampType(3), "CAST(NULL AS TIMESTAMP)")
+ .addRoundTrip("timestamp", "TIMESTAMP '2019-03-18 10:01:17.987'", createTimestampType(3), "TIMESTAMP '2019-03-18 10:01:17.987'")
+ .addRoundTrip("timestamp", "TIMESTAMP '2013-03-11 17:30:15.123'", createTimestampType(3), "TIMESTAMP '2013-03-11 17:30:15.123'")
+ .addRoundTrip("timestamp", "TIMESTAMP '2018-10-28 01:33:17.456'", createTimestampType(3), "TIMESTAMP '2018-10-28 01:33:17.456'")
+ .addRoundTrip("timestamp", "TIMESTAMP '2018-10-28 03:33:33.333'", createTimestampType(3), "TIMESTAMP '2018-10-28 03:33:33.333'")
+ .addRoundTrip("timestamp", "TIMESTAMP '1970-01-01 00:13:42.000'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:13:42.000'")
+ .addRoundTrip("timestamp", "TIMESTAMP '2018-04-01 03:13:55.123'", createTimestampType(3), "TIMESTAMP '2018-04-01 03:13:55.123'")
+ .addRoundTrip("timestamp", "TIMESTAMP '2020-09-27 12:34:56.999'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.999'")
+ .addRoundTrip("timestamp", "TIMESTAMP '2018-03-25 03:17:17.000'", createTimestampType(3), "TIMESTAMP '2018-03-25 03:17:17.000'")
+ .addRoundTrip("timestamp", "TIMESTAMP '1986-01-01 00:13:07.000'", createTimestampType(3), "TIMESTAMP '1986-01-01 00:13:07.000'")
+ .addRoundTrip("timestamp(6)", "TIMESTAMP '2013-03-11 17:30:15.123456'", createTimestampType(6), "TIMESTAMP '2013-03-11 17:30:15.123456'")
+ .addRoundTrip("timestamp(9)", "TIMESTAMP '2013-03-11 17:30:15.123456789'", createTimestampType(9), "TIMESTAMP '2013-03-11 17:30:15.123456789'")
+ .addRoundTrip("timestamp(1)", "TIMESTAMP '2016-08-19 19:28:05.0'", createTimestampType(1), "TIMESTAMP '2016-08-19 19:28:05.0'")
+ .addRoundTrip("timestamp(2)", "TIMESTAMP '2016-08-19 19:28:05.01'", createTimestampType(2), "TIMESTAMP '2016-08-19 19:28:05.01'")
+ .addRoundTrip("timestamp", "TIMESTAMP '3030-03-03 12:34:56.123'", createTimestampType(3), "TIMESTAMP '3030-03-03 12:34:56.123'")
+ .addRoundTrip("timestamp(5)", "TIMESTAMP '3030-03-03 12:34:56.12345'", createTimestampType(5), "TIMESTAMP '3030-03-03 12:34:56.12345'")
+ .addRoundTrip("timestamp(9)", "TIMESTAMP '3030-03-03 12:34:56.123456789'", createTimestampType(9), "TIMESTAMP '3030-03-03 12:34:56.123456789'")
+ .addRoundTrip("timestamp", "TIMESTAMP '3030-03-03 12:34:56.123'", createTimestampType(3), "TIMESTAMP '3030-03-03 12:34:56.123'")
+ .addRoundTrip("timestamp(5)", "TIMESTAMP '3030-03-03 12:34:56.12345'", createTimestampType(5), "TIMESTAMP '3030-03-03 12:34:56.12345'")
+ .addRoundTrip("timestamp(6)", "TIMESTAMP '3030-03-03 12:34:56.123456'", createTimestampType(6), "TIMESTAMP '3030-03-03 12:34:56.123456'")
+ .addRoundTrip("timestamp(7)", "TIMESTAMP '3030-03-03 12:34:56.1234567'", createTimestampType(7), "TIMESTAMP '3030-03-03 12:34:56.1234567'")
+ .addRoundTrip("timestamp(8)", "TIMESTAMP '3030-03-03 12:34:56.12345678'", createTimestampType(8), "TIMESTAMP '3030-03-03 12:34:56.12345678'")
+ .addRoundTrip("timestamp(9)", "TIMESTAMP '3030-03-03 12:34:56.123456789'", createTimestampType(9), "TIMESTAMP '3030-03-03 12:34:56.123456789'")
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '2017-07-01'", createTimestampType(0), "TIMESTAMP '2017-07-01'") // summer on northern hemisphere (possible DST)
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '2017-01-01'", createTimestampType(0), "TIMESTAMP '2017-01-01'") // winter on northern hemisphere (possible DST on southern hemisphere)
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '1970-01-01'", createTimestampType(0), "TIMESTAMP '1970-01-01'") // change forward at midnight in JVM
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '1983-04-01'", createTimestampType(0), "TIMESTAMP '1983-04-01'") // change forward at midnight in Vilnius
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '1983-10-01'", createTimestampType(0), "TIMESTAMP '1983-10-01'") // change backward at midnight in Vilnius
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '9999-12-31'", createTimestampType(0), "TIMESTAMP '9999-12-31'") // max value in Exasol
+ //test cases for timestamp with zero precision and with non-zero seconds
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '2017-07-01 00:00:01'", createTimestampType(0), "TIMESTAMP '2017-07-01 00:00:01'") // summer on northern hemisphere (possible DST)
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '2017-01-01 00:00:02'", createTimestampType(0), "TIMESTAMP '2017-01-01 00:00:02'") // winter on northern hemisphere (possible DST on southern hemisphere)
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '1970-01-01 00:00:03'", createTimestampType(0), "TIMESTAMP '1970-01-01 00:00:03'") // change forward at midnight in JVM
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '1983-04-01 00:00:04'", createTimestampType(0), "TIMESTAMP '1983-04-01 00:00:04'") // change forward at midnight in Vilnius
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '1983-10-01 00:00:05'", createTimestampType(0), "TIMESTAMP '1983-10-01 00:00:05'") // change backward at midnight in Vilnius
+ .addRoundTrip("timestamp(0)", "TIMESTAMP '9999-12-31 00:00:59'", createTimestampType(0), "TIMESTAMP '9999-12-31 00:00:59'") // max value in Exasol
+ .execute(getQueryRunner(), session, exasolCreateAndInsert(TEST_SCHEMA + "." + "test_timestamp"));
+ }
+
+ @Test
+ void testTimestampWithTimeZone()
+ {
+ testTimestampWithTimeZone(UTC);
+ testTimestampWithTimeZone(jvmZone);
+ // using two non-JVM zones so that we don't need to worry what Exasol system zone is
+ testTimestampWithTimeZone(vilnius);
+ testTimestampWithTimeZone(kathmandu);
+ testTimestampWithTimeZone(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId());
+ }
+
+ /**
+ *
+ * Exasol {@code TIMESTAMP WITH LOCAL TIME ZONE} does not store any time zone information in the database.
+ * In Trino, it is mapped to {@code TIMESTAMP WITH TIME ZONE} with the hardcoded JVM time zone
+ *
+ *
+ *
+ * This limitation imposes some rules and restrictions on the tests:
+ *
+ * - Both {@code inputLiteral} and {@code expectedLiteral} are interpreted as JVM time zone strings.
+ * - In the Trino testing environment, the JVM time zone is hardcoded to {@code America/Bahia_Banderas}.
+ * - When saved {@code inputLiteral} is read by Trino, it must be equal to the {@code expectedLiteral}, interpreted as JVM time zone value
+ *
+ *
+ *
+ * See for more details: https://docs.exasol.com/db/latest/sql_references/data_types/datatypedetails.htm
+ *
+ */
+ private void testTimestampWithTimeZone(ZoneId sessionZone)
+ {
+ Session session = Session.builder(getSession())
+ .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId()))
+ .build();
+
+ SqlDataTypeTest.create()
+ .addRoundTrip("timestamp with local time zone", "NULL", createTimestampWithTimeZoneType(3), "CAST(NULL AS TIMESTAMP WITH TIME ZONE)")
+
+ // timestamp with precision 3 examples
+ .addRoundTrip("timestamp with local time zone", timestampWithTimeZoneInput("2019-03-18 10:01:17.123"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2019-03-18 10:01:17.123"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("2018-10-27 19:33:17.456"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2018-10-27 19:33:17.456"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("2018-10-28 03:33:33.333"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2018-10-28 03:33:33.333"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:13:42.000"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("1970-01-01 00:13:42.000"))
+
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("2018-04-01 03:13:55.123"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2018-04-01 03:13:55.123"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("2020-09-27 12:34:56.999"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2020-09-27 12:34:56.999"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("2018-03-25 03:17:17.000"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2018-03-25 03:17:17.000"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("1986-01-01 00:13:07.000"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("1986-01-01 00:13:07.000"))
+
+ // timestamp with precision 6-9 examples
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("2019-03-18 10:01:17.987654"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("2019-03-18 10:01:17.987654"))
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("2018-10-28 01:33:17.456789"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("2018-10-28 01:33:17.456789"))
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("2018-10-28 03:33:33.333333"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("2018-10-28 03:33:33.333333"))
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:13:42.000000"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("1970-01-01 00:13:42.000000"))
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("2018-04-01 03:13:55.123456"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("2018-04-01 03:13:55.123456"))
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("2018-03-25 03:17:17.000000"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("2018-03-25 03:17:17.000000"))
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("1986-01-01 00:13:07.000000"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("1986-01-01 00:13:07.000000"))
+ .addRoundTrip("timestamp(7) with local time zone", timestampWithTimeZoneInput("1986-01-01 00:13:07.1234567"), createTimestampWithTimeZoneType(7), timestampWithTimeZoneExpected("1986-01-01 00:13:07.1234567"))
+ .addRoundTrip("timestamp(8) with local time zone", timestampWithTimeZoneInput("1986-01-01 00:13:07.12345678"), createTimestampWithTimeZoneType(8), timestampWithTimeZoneExpected("1986-01-01 00:13:07.12345678"))
+ .addRoundTrip("timestamp(9) with local time zone", timestampWithTimeZoneInput("1986-01-01 00:13:07.123456789"), createTimestampWithTimeZoneType(9), timestampWithTimeZoneExpected("1986-01-01 00:13:07.123456789"))
+
+ // tests for other precisions (0-5 and some 1's)
+ .addRoundTrip("timestamp(0) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01"), createTimestampWithTimeZoneType(0), timestampWithTimeZoneExpected("1970-01-01 00:00:01"))
+ .addRoundTrip("timestamp(1) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01.1"), createTimestampWithTimeZoneType(1), timestampWithTimeZoneExpected("1970-01-01 00:00:01.1"))
+ .addRoundTrip("timestamp(1) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01.9"), createTimestampWithTimeZoneType(1), timestampWithTimeZoneExpected("1970-01-01 00:00:01.9"))
+ .addRoundTrip("timestamp(2) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01.12"), createTimestampWithTimeZoneType(2), timestampWithTimeZoneExpected("1970-01-01 00:00:01.12"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01.123"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("1970-01-01 00:00:01.123"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01.999"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("1970-01-01 00:00:01.999"))
+ .addRoundTrip("timestamp(4) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01.1234"), createTimestampWithTimeZoneType(4), timestampWithTimeZoneExpected("1970-01-01 00:00:01.1234"))
+ .addRoundTrip("timestamp(5) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:01.12345"), createTimestampWithTimeZoneType(5), timestampWithTimeZoneExpected("1970-01-01 00:00:01.12345"))
+ .addRoundTrip("timestamp(1) with local time zone", timestampWithTimeZoneInput("2020-09-27 12:34:56.1"), createTimestampWithTimeZoneType(1), timestampWithTimeZoneExpected("2020-09-27 12:34:56.1"))
+ .addRoundTrip("timestamp(1) with local time zone", timestampWithTimeZoneInput("2020-09-27 12:34:56.9"), createTimestampWithTimeZoneType(1), timestampWithTimeZoneExpected("2020-09-27 12:34:56.9"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("2020-09-27 12:34:56.123"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2020-09-27 12:34:56.123"))
+ .addRoundTrip("timestamp(3) with local time zone", timestampWithTimeZoneInput("2020-09-27 12:34:56.999"), createTimestampWithTimeZoneType(3), timestampWithTimeZoneExpected("2020-09-27 12:34:56.999"))
+ .addRoundTrip("timestamp(6) with local time zone", timestampWithTimeZoneInput("2020-09-27 12:34:56.123456"), createTimestampWithTimeZoneType(6), timestampWithTimeZoneExpected("2020-09-27 12:34:56.123456"))
+
+ //test cases for timestamp with zero precision and with non-zero seconds
+ .addRoundTrip("timestamp(0) with local time zone", timestampWithTimeZoneInput("2017-07-01 00:00:01"), createTimestampWithTimeZoneType(0), timestampWithTimeZoneExpected("2017-07-01 00:00:01")) // summer on northern hemisphere (possible DST)
+ .addRoundTrip("timestamp(0) with local time zone", timestampWithTimeZoneInput("2017-01-01 00:00:02"), createTimestampWithTimeZoneType(0), timestampWithTimeZoneExpected("2017-01-01 00:00:02")) // winter on northern hemisphere (possible DST on southern hemisphere)
+ .addRoundTrip("timestamp(0) with local time zone", timestampWithTimeZoneInput("1970-01-01 00:00:03"), createTimestampWithTimeZoneType(0), timestampWithTimeZoneExpected("1970-01-01 00:00:03")) // change forward at midnight in JVM
+ .addRoundTrip("timestamp(0) with local time zone", timestampWithTimeZoneInput("1983-04-01 00:00:04"), createTimestampWithTimeZoneType(0), timestampWithTimeZoneExpected("1983-04-01 00:00:04")) // change forward at midnight in Vilnius
+ .addRoundTrip("timestamp(0) with local time zone", timestampWithTimeZoneInput("1983-10-01 00:00:05"), createTimestampWithTimeZoneType(0), timestampWithTimeZoneExpected("1983-10-01 00:00:05")) // change backward at midnight in Vilnius
+ .addRoundTrip("timestamp(0) with local time zone", timestampWithTimeZoneInput("9999-12-31 00:00:59"), createTimestampWithTimeZoneType(0), timestampWithTimeZoneExpected("9999-12-31 00:00:59")) // max value in Exasol
+
+ .execute(getQueryRunner(), session, exasolCreateAndInsert(TEST_SCHEMA + "." + "test_timestamp"));
+ }
+
+ private String timestampWithTimeZoneInput(String jvmTimeZoneString)
+ {
+ return "TIMESTAMP '%s'".formatted(jvmTimeZoneString);
+ }
+
+ private String timestampWithTimeZoneExpected(String jvmTimeZoneString)
+ {
+ return "TIMESTAMP '%s %s'".formatted(jvmTimeZoneString, jvmZone.getId());
+ }
+
+ @Test
+ // See for more details: https://docs.exasol.com/db/latest/sql_references/data_types/datatypedetails.htm
+ void testUnsupportedTimestampValues()
+ {
+ // Below minimum supported TIMESTAMP value (must be >= 0001-01-01)
+ testUnsupportedInsertValue(
+ "TIMESTAMP",
+ "TIMESTAMP '10000-01-01 00:00:00.000000'",
+ "data exception - invalid character value for cast; Value: '10000-01-01 00:00:00.000000'");
+
+ // Above maximum supported TIMESTAMP value (must be <= 9999-12-31)
+ testUnsupportedInsertValue(
+ "TIMESTAMP",
+ "TIMESTAMP '0000-12-31 23:59:59.999999'",
+ "data exception - invalid date value; Value: '0000-12-31 23:59:59.999999'");
+
+ // Exceeds TIMESTAMP maximum supported fractional seconds precision (9 digits)
+ testUnsupportedInsertValue(
+ "TIMESTAMP",
+ "TIMESTAMP '2024-01-01 12:34:56.1234567890'",
+ "data exception - invalid character value for cast; Value: '2024-01-01 12:34:56.1234567890'");
+
+ // Negative precisions are not supported
+ testUnsupportedDefinition(
+ "TIMESTAMP(-1)",
+ "syntax error, unexpected '-', expecting UNSIGNED_INTEGER");
+ }
+
+ @Test
+ // See for more details: https://docs.exasol.com/db/latest/sql_references/data_types/datatypedetails.htm
+ void testUnsupportedTimestampWithLocalTimeZoneValues()
+ {
+ testUnsupportedDstGapJvmTimeZoneValue("2018-04-01 02:13:55.123", 3);
+ testUnsupportedDstGapJvmTimeZoneValue("2018-04-01 02:13:55.123456", 6);
+
+ // Below minimum supported TIMESTAMP WITH LOCAL TIME ZONE value (must be >= 0001-01-01)
+ testUnsupportedInsertValue(
+ "TIMESTAMP WITH LOCAL TIME ZONE",
+ "TIMESTAMP '10000-01-01 00:00:00.000000'",
+ "data exception - invalid character value for cast; Value: '10000-01-01 00:00:00.000000'");
+
+ // Above maximum supported TIMESTAMP WITH LOCAL TIME ZONE value (must be <= 9999-12-31)
+ testUnsupportedInsertValue(
+ "TIMESTAMP WITH LOCAL TIME ZONE",
+ "TIMESTAMP '0000-12-31 23:59:59.999999'",
+ "data exception - invalid date value; Value: '0000-12-31 23:59:59.999999'");
+
+ // Exceeds TIMESTAMP WITH LOCAL TIME ZONE maximum supported fractional seconds precision (9 digits)
+ testUnsupportedInsertValue(
+ "TIMESTAMP WITH LOCAL TIME ZONE",
+ "TIMESTAMP '2024-01-01 12:34:56.1234567890'",
+ "data exception - invalid character value for cast; Value: '2024-01-01 12:34:56.1234567890'");
+
+ // Negative precisions are not supported
+ testUnsupportedDefinition(
+ "TIMESTAMP(-1) WITH LOCAL TIME ZONE",
+ "syntax error, unexpected '-', expecting UNSIGNED_INTEGER");
+ }
+
+ private void testUnsupportedDstGapJvmTimeZoneValue(String dstGapJvmTimeZoneString, int precision)
+ {
+ try (TestTable table = new TestTable(onRemoteDatabase(), TestingExasolServer.TEST_SCHEMA + ".timestamp_dst_gap",
+ "(col TIMESTAMP(%d) WITH LOCAL TIME ZONE)".formatted(precision))) {
+ // Exasol successfully resolves dst gap value and saves it to database
+ onRemoteDatabase().execute(format("INSERT INTO %s (col) VALUES (%s)", table.getName(),
+ "TO_TIMESTAMP('%s', 'YYYY-MM-DD HH24:MI:SS.FF%d')".formatted(dstGapJvmTimeZoneString, precision)));
+ assertThat(exasolServer.getSingleResult(
+ "SELECT count(*) FROM %s WHERE col = TO_TIMESTAMP('%s', 'YYYY-MM-DD HH24:MI:SS.FF%d')".formatted(table.getName(), dstGapJvmTimeZoneString, precision),
+ String.class))
+ .matches("1");
+
+ //Trino query throws exception, because dst gap value is not valid for 'America/Bahia_Banderas' time zone
+ assertThat(query(
+ "SELECT count(*) FROM %s WHERE col = TIMESTAMP '%s %s'".formatted(table.getName(), dstGapJvmTimeZoneString, jvmZone.getId())))
+ .failure().hasMessageEndingWith("America/Bahia_Banderas' is not a valid TIMESTAMP literal");
+ }
+ }
+
// See for more details: https://docs.exasol.com/saas/microcontent/Resources/MicroContent/general/hash-data-type.htm
@Test
void testHashtype()
@@ -312,36 +548,36 @@ void testHashtype()
void testUnsupportedHashTypeDefinitions()
{
// Too few bytes (< 1)
- testUnsupportedHashTypeDefinition(
+ testUnsupportedDefinition(
"HASHTYPE(0 BYTE)",
"the given size of HASHTYPE is too small. A minimum of 1 bytes are required");
// Too many bytes (> 1024)
- testUnsupportedHashTypeDefinition(
+ testUnsupportedDefinition(
"HASHTYPE(1025 BYTE)",
"the given size of HASHTYPE is too large. At most 1024 bytes are allowed");
// Too few bits (< 8)
- testUnsupportedHashTypeDefinition(
+ testUnsupportedDefinition(
"HASHTYPE(7 BIT)",
"the given size of HASHTYPE is too small. A minimum of 8 bits are required");
// Too many bits (> 8192)
- testUnsupportedHashTypeDefinition(
+ testUnsupportedDefinition(
"HASHTYPE(8193 BIT)",
"the given size of HASHTYPE is too large. At most 8192 bits are allowed");
// Bits not divisible by 8
- testUnsupportedHashTypeDefinition(
+ testUnsupportedDefinition(
"HASHTYPE(9 BIT)",
"Bit size of HASHTYPE has to be a multiple of 8");
}
- private void testUnsupportedHashTypeDefinition(
+ private void testUnsupportedDefinition(
String exasolType,
String expectedException)
{
- String tableName = "test_unsupported_hashtype_" + randomNameSuffix();
+ String tableName = "test_unsupported_definition_" + randomNameSuffix();
assertExasolSqlQueryFails(
"CREATE TABLE %s.%s (col %s)".formatted(TEST_SCHEMA, tableName, exasolType),
expectedException);
@@ -352,43 +588,43 @@ private void testUnsupportedHashTypeDefinition(
void testUnsupportedHashTypeInsertValues()
{
// Invalid hex character
- testUnsupportedHashTypeInsertValue(
+ testUnsupportedInsertValue(
"HASHTYPE(4 BYTE)",
"'GGGGGGGG'",
"data exception - Invalid hash format");
// Too short for declared size (expecting 4 bytes = 8 hex chars, got 6)
- testUnsupportedHashTypeInsertValue(
+ testUnsupportedInsertValue(
"HASHTYPE(4 BYTE)",
"'AABBCC'",
"data exception - Invalid hash format");
// Too short for declared size (expecting 16 bytes = 32 hex chars, got 31)
- testUnsupportedHashTypeInsertValue(
+ testUnsupportedInsertValue(
"HASHTYPE(16 BYTE)",
"'550e8400-e29b-11d4-a716-44665544000'",
"data exception - Invalid hash format");
// Too long for declared size (expecting 4 bytes = 8 hex chars, got 10)
- testUnsupportedHashTypeInsertValue(
+ testUnsupportedInsertValue(
"HASHTYPE(4 BYTE)",
"'AABBCCDDEE'",
"data exception - Invalid hash format");
// Unexpected symbol inside
- testUnsupportedHashTypeInsertValue(
+ testUnsupportedInsertValue(
"HASHTYPE(4 BYTE)",
"'AABB-CCZZ'",
"data exception - Invalid hash format");
// Parentheses instead of curly brackets
- testUnsupportedHashTypeInsertValue(
+ testUnsupportedInsertValue(
"HASHTYPE(4 BYTE)",
"'(AABB-CCCC)'",
"data exception - Invalid hash format");
}
- private void testUnsupportedHashTypeInsertValue(
+ private void testUnsupportedInsertValue(
String exasolType,
String inputLiteral,
String expectedException)
diff --git a/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestingExasolServer.java b/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestingExasolServer.java
index b3021a8b2f43..b70aa76e3e96 100644
--- a/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestingExasolServer.java
+++ b/plugin/trino-exasol/src/test/java/io/trino/plugin/exasol/TestingExasolServer.java
@@ -24,6 +24,7 @@
import java.io.UncheckedIOException;
import java.sql.Connection;
import java.sql.DriverManager;
+import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Properties;
@@ -87,6 +88,27 @@ public void execute(@Language("SQL") String sql, String user, String password)
}
}
+ public T getSingleResult(@Language("SQL") String sql, Class resultClass)
+ {
+ return getSingleResult(sql, container.getUsername(), container.getPassword(), resultClass);
+ }
+
+ public T getSingleResult(@Language("SQL") String sql, String user, String password, Class resultClass)
+ {
+ try (Connection connection = DriverManager.getConnection(getJdbcUrl(), getProperties(user, password));
+ Statement statement = connection.createStatement()) {
+ ResultSet rs = statement.executeQuery(sql);
+ if (rs.next()) {
+ T value = rs.getObject(1, resultClass);
+ return value;
+ }
+ return null;
+ }
+ catch (SQLException e) {
+ throw new RuntimeException("Failed to execute statement '" + sql + "'", e);
+ }
+ }
+
public JdbcSqlExecutor getSqlExecutor()
{
return new JdbcSqlExecutor(getJdbcUrl(), getProperties(TEST_USER, TEST_PASSWORD));