diff --git a/docs/src/main/sphinx/connector/oracle.rst b/docs/src/main/sphinx/connector/oracle.rst index 247ce49190d8..6704ef37c715 100644 --- a/docs/src/main/sphinx/connector/oracle.rst +++ b/docs/src/main/sphinx/connector/oracle.rst @@ -203,7 +203,7 @@ Trino data type mapping: - ``TIMESTAMP(0)`` - See :ref:`datetime mapping` * - ``TIMESTAMP(p)`` - - ``TIMESTAMP`` + - ``TIMESTAMP(p)`` - See :ref:`datetime mapping` * - ``TIMESTAMP(p) WITH TIME ZONE`` - ``TIMESTAMP WITH TIME ZONE`` @@ -300,8 +300,8 @@ For Oracle ``NUMBER`` (without precision and scale), you can change Mapping datetime types ^^^^^^^^^^^^^^^^^^^^^^ -Selecting a timestamp with fractional second precision (``p``) greater than 3 -truncates the fractional seconds to three digits instead of rounding it. +Writing a timestamp with fractional second precision (``p``) greater than 9 +rounds the fractional seconds to nine digits. Oracle ``DATE`` type stores hours, minutes, and seconds, so it is mapped to Trino ``TIMESTAMP(0)``. diff --git a/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java b/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java index 64e2546f6994..82926c3c01e4 100644 --- a/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java +++ b/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java @@ -34,6 +34,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.RemoteTableName; import io.trino.plugin.jdbc.SliceWriteFunction; @@ -63,6 +65,8 @@ import io.trino.spi.type.CharType; import io.trino.spi.type.DecimalType; import io.trino.spi.type.Decimals; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.TimestampType; import io.trino.spi.type.Type; import io.trino.spi.type.VarcharType; import oracle.jdbc.OraclePreparedStatement; @@ -80,6 +84,8 @@ import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; import java.util.List; import java.util.Map; import java.util.Optional; @@ -97,6 +103,8 @@ import static io.trino.plugin.jdbc.StandardColumnMappings.bigintColumnMapping; import static io.trino.plugin.jdbc.StandardColumnMappings.bigintWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.charReadFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.fromLongTrinoTimestamp; +import static io.trino.plugin.jdbc.StandardColumnMappings.fromTrinoTimestamp; import static io.trino.plugin.jdbc.StandardColumnMappings.integerWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalReadFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalWriteFunction; @@ -104,6 +112,8 @@ import static io.trino.plugin.jdbc.StandardColumnMappings.shortDecimalWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.smallintWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.toLongTrinoTimestamp; +import static io.trino.plugin.jdbc.StandardColumnMappings.toTrinoTimestamp; import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.varcharWriteFunction; import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.getUnsupportedTypeHandling; @@ -123,12 +133,11 @@ 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.MAX_SHORT_PRECISION; 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_MILLIS; -import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_SECOND; -import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MICROSECOND; import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; @@ -150,6 +159,7 @@ public class OracleClient { public static final int ORACLE_MAX_LIST_EXPRESSIONS = 1000; + private static final int MAX_ORACLE_TIMESTAMP_PRECISION = 9; private static final int MAX_BYTES_PER_CHAR = 4; private static final int ORACLE_VARCHAR2_MAX_BYTES = 4000; @@ -164,7 +174,13 @@ public class OracleClient private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd"); private static final DateTimeFormatter TIMESTAMP_SECONDS_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss"); - private static final DateTimeFormatter TIMESTAMP_MILLIS_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss.SSS"); + + private static final DateTimeFormatter TIMESTAMP_NANO_OPTIONAL_FORMATTER = new DateTimeFormatterBuilder() + .appendPattern("uuuu-MM-dd HH:mm:ss") + .optionalStart() + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + .optionalEnd() + .toFormatter(); private static final Set INTERNAL_SCHEMAS = ImmutableSet.builder() .add("ctxsys") @@ -385,7 +401,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect if (jdbcTypeName.equalsIgnoreCase("date")) { return Optional.of(ColumnMapping.longMapping( TIMESTAMP_SECONDS, - oracleTimestampReadFunction(), + oracleTimestampReadFunction(TIMESTAMP_SECONDS), trinoTimestampToOracleDateWriteFunction(), FULL_PUSHDOWN)); } @@ -482,11 +498,8 @@ else if (precision > Decimals.MAX_PRECISION || actualPrecision <= 0) { DISABLE_PUSHDOWN)); case OracleTypes.TIMESTAMP: - return Optional.of(ColumnMapping.longMapping( - TIMESTAMP_MILLIS, - oracleTimestampReadFunction(), - trinoTimestampToOracleTimestampWriteFunction(), - FULL_PUSHDOWN)); + int timestampPrecision = typeHandle.getRequiredDecimalDigits(); + return Optional.of(oracleTimestampColumnMapping(createTimestampType(timestampPrecision))); case OracleTypes.TIMESTAMPTZ: return Optional.of(oracleTimestampWithTimeZoneColumnMapping()); } @@ -496,6 +509,22 @@ else if (precision > Decimals.MAX_PRECISION || actualPrecision <= 0) { return Optional.empty(); } + private static ColumnMapping oracleTimestampColumnMapping(TimestampType timestampType) + { + if (timestampType.isShort()) { + return ColumnMapping.longMapping( + timestampType, + oracleTimestampReadFunction(timestampType), + oracleTimestampWriteFunction(timestampType), + FULL_PUSHDOWN); + } + return ColumnMapping.objectMapping( + timestampType, + oracleLongTimestampReadFunction(timestampType), + oracleLongTimestampWriteFunction(timestampType), + FULL_PUSHDOWN); + } + @Override public Optional implementAggregation(ConnectorSession session, AggregateFunction aggregate, Map assignments) { @@ -584,24 +613,57 @@ public void setNull(PreparedStatement statement, int index) }; } - public static LongWriteFunction trinoTimestampToOracleTimestampWriteFunction() + private static ObjectWriteFunction oracleLongTimestampWriteFunction(TimestampType timestampType) + { + int precision = timestampType.getPrecision(); + verifyLongTimestampPrecision(timestampType); + + return new ObjectWriteFunction() { + @Override + public Class getJavaType() + { + return LongTimestamp.class; + } + + @Override + public void set(PreparedStatement statement, int index, Object value) + throws SQLException + { + LocalDateTime timestamp = fromLongTrinoTimestamp((LongTimestamp) value, precision); + statement.setString(index, TIMESTAMP_NANO_OPTIONAL_FORMATTER.format(timestamp)); + } + + @Override + public String getBindExpression() + { + return getOracleBindExpression(precision); + } + + @Override + public void setNull(PreparedStatement statement, int index) + throws SQLException + { + statement.setNull(index, Types.VARCHAR); + } + }; + } + + private static LongWriteFunction oracleTimestampWriteFunction(TimestampType timestampType) { return new LongWriteFunction() { @Override public String getBindExpression() { - return "TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS.FF')"; + return getOracleBindExpression(timestampType.getPrecision()); } @Override - public void set(PreparedStatement statement, int index, long utcMillis) + public void set(PreparedStatement statement, int index, long epochMicros) throws SQLException { - long epochSecond = floorDiv(utcMillis, MICROSECONDS_PER_SECOND); - int nanoFraction = floorMod(utcMillis, MICROSECONDS_PER_SECOND) * NANOSECONDS_PER_MICROSECOND; - LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(epochSecond, nanoFraction, ZoneOffset.UTC); - statement.setString(index, TIMESTAMP_MILLIS_FORMATTER.format(localDateTime)); + LocalDateTime timestamp = fromTrinoTimestamp(epochMicros); + statement.setString(index, TIMESTAMP_NANO_OPTIONAL_FORMATTER.format(timestamp)); } @Override @@ -613,7 +675,19 @@ public void setNull(PreparedStatement statement, int index) }; } - private static LongReadFunction oracleTimestampReadFunction() + private static String getOracleBindExpression(int precision) + { + if (precision == 0) { + return "TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS')"; + } + if (precision <= 2) { + return "TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS.FF')"; + } + + return format("TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS.FF%d')", precision); + } + + private static LongReadFunction oracleTimestampReadFunction(TimestampType timestampType) { return (resultSet, columnIndex) -> { LocalDateTime timestamp = resultSet.getObject(columnIndex, LocalDateTime.class); @@ -621,10 +695,32 @@ private static LongReadFunction oracleTimestampReadFunction() if (timestamp.getYear() <= 0) { timestamp = timestamp.minusYears(1); } - return timestamp.toInstant(ZoneOffset.UTC).toEpochMilli() * MICROSECONDS_PER_MILLISECOND; + return toTrinoTimestamp(timestampType, timestamp); }; } + private static ObjectReadFunction oracleLongTimestampReadFunction(TimestampType timestampType) + { + verifyLongTimestampPrecision(timestampType); + return ObjectReadFunction.of( + LongTimestamp.class, + (resultSet, columnIndex) -> { + LocalDateTime timestamp = resultSet.getObject(columnIndex, LocalDateTime.class); + // Adjust years when the value is B.C. dates because Oracle returns +1 year unless converting to string in their server side + if (timestamp.getYear() <= 0) { + timestamp = timestamp.minusYears(1); + } + return toLongTrinoTimestamp(timestampType, timestamp); + }); + } + + private static void verifyLongTimestampPrecision(TimestampType timestampType) + { + int precision = timestampType.getPrecision(); + checkArgument(precision > MAX_SHORT_PRECISION && precision <= MAX_ORACLE_TIMESTAMP_PRECISION, + "Precision is out of range: %s", precision); + } + public static ColumnMapping oracleTimestampWithTimeZoneColumnMapping() { return ColumnMapping.longMapping( @@ -705,13 +801,18 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) } return WriteMapping.objectMapping(dataType, longDecimalWriteFunction(decimalType)); } - if (type.equals(TIMESTAMP_SECONDS)) { - // Specify 'date' instead of 'timestamp(0)' to propagate the type in case of CTAS from date columns - // Oracle date stores year, month, day, hour, minute, seconds, but not second fraction - return WriteMapping.longMapping("date", trinoTimestampToOracleDateWriteFunction()); - } - if (type.equals(TIMESTAMP_MILLIS)) { - return WriteMapping.longMapping("timestamp(3)", trinoTimestampToOracleTimestampWriteFunction()); + if (type instanceof TimestampType timestampType) { + if (type.equals(TIMESTAMP_SECONDS)) { + // Specify 'date' instead of 'timestamp(0)' to propagate the type in case of CTAS from date columns + // Oracle date stores year, month, day, hour, minute, seconds, but not second fraction + return WriteMapping.longMapping("date", trinoTimestampToOracleDateWriteFunction()); + } + int precision = min(timestampType.getPrecision(), MAX_ORACLE_TIMESTAMP_PRECISION); + String dataType = format("timestamp(%d)", precision); + if (timestampType.isShort()) { + return WriteMapping.longMapping(dataType, oracleTimestampWriteFunction(timestampType)); + } + return WriteMapping.objectMapping(dataType, oracleLongTimestampWriteFunction(createTimestampType(precision))); } WriteMapping writeMapping = WRITE_MAPPINGS.get(type); if (writeMapping != null) { diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java index 0142adc41591..55ddb20ab1e5 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java @@ -52,7 +52,9 @@ import static io.trino.spi.type.RealType.REAL; import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey; import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; 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_MILLIS; import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; @@ -727,12 +729,176 @@ public void testTimestamp(ZoneId sessionZone) .addRoundTrip("timestamp", timestampDataType(3).toLiteral(timeGapInKathmandu), TIMESTAMP_MILLIS, timestampDataType(3).toLiteral(timeGapInKathmandu)) // max value in Oracle .addRoundTrip("timestamp", "TIMESTAMP '9999-12-31 00:00:00.000'", TIMESTAMP_MILLIS, "TIMESTAMP '9999-12-31 00:00:00.000'") - .execute(getQueryRunner(), session, oracleCreateAndInsert("test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAsSelect("test_timestamp")) .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp")); } + @Test(dataProvider = "sessionZonesDataProvider") + public void testTimestampNanos(ZoneId sessionZone) + { + Session session = Session.builder(getSession()) + .setTimeZoneKey(getTimeZoneKey(sessionZone.getId())) + .build(); + + SqlDataTypeTest.create() + // min value in Oracle + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '-4712-01-01 00:00:00.000000000'", TIMESTAMP_NANOS, "TIMESTAMP '-4712-01-01 00:00:00.000000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '-0001-01-01 00:00:00.000000000'", TIMESTAMP_NANOS, "TIMESTAMP '-0001-01-01 00:00:00.000000000'") + // day before and after julian->gregorian calendar switch + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1582-10-04 00:00:00.000000000'", TIMESTAMP_NANOS, "TIMESTAMP '1582-10-04 00:00:00.000000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1582-10-15 00:00:00.000000000'", TIMESTAMP_NANOS, "TIMESTAMP '1582-10-15 00:00:00.000000000'") + // before epoch + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1958-01-01 13:18:03.123123123'", TIMESTAMP_NANOS, "TIMESTAMP '1958-01-01 13:18:03.123123123'") + // after epoch + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2019-03-18 10:01:17.987987987'", TIMESTAMP_NANOS, "TIMESTAMP '2019-03-18 10:01:17.987987987'") + // epoch, epoch also is a gap in JVM zone + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.000000000'", TIMESTAMP_NANOS, "TIMESTAMP '1970-01-01 00:00:00.000000000'") + .addRoundTrip("TIMESTAMP(9)", timestampDataType(9).toLiteral(timeDoubledInJvmZone), TIMESTAMP_NANOS, timestampDataType(9).toLiteral(timeDoubledInJvmZone)) + .addRoundTrip("TIMESTAMP(9)", timestampDataType(9).toLiteral(timeDoubledInVilnius), TIMESTAMP_NANOS, timestampDataType(9).toLiteral(timeDoubledInVilnius)) + .addRoundTrip("TIMESTAMP(9)", timestampDataType(9).toLiteral(timeGapInJvmZone1), TIMESTAMP_NANOS, timestampDataType(9).toLiteral(timeGapInJvmZone1)) + .addRoundTrip("TIMESTAMP(9)", timestampDataType(9).toLiteral(timeGapInJvmZone2), TIMESTAMP_NANOS, timestampDataType(9).toLiteral(timeGapInJvmZone2)) + .addRoundTrip("TIMESTAMP(9)", timestampDataType(9).toLiteral(timeGapInVilnius), TIMESTAMP_NANOS, timestampDataType(9).toLiteral(timeGapInVilnius)) + .addRoundTrip("TIMESTAMP(9)", timestampDataType(9).toLiteral(timeGapInKathmandu), TIMESTAMP_NANOS, timestampDataType(9).toLiteral(timeGapInKathmandu)) + // max value in Oracle + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '9999-12-31 00:00:00.000000000'", TIMESTAMP_NANOS, "TIMESTAMP '9999-12-31 00:00:00.000000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '9999-12-31 23:59:59.999999999'", TIMESTAMP_NANOS, "TIMESTAMP '9999-12-31 23:59:59.999999999'") + .execute(getQueryRunner(), session, oracleCreateAndInsert("test_timestamp_nano")) + .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp_nano")) + .execute(getQueryRunner(), session, trinoCreateAsSelect("test_timestamp_nano")) + .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp_nano")); + } + + @Test(dataProvider = "sessionZonesDataProvider") + public void testTimestampAllPrecisions(ZoneId sessionZone) + { + SqlDataTypeTest tests = SqlDataTypeTest.create() + // before epoch + .addRoundTrip("TIMESTAMP '1958-01-01 13:18:03.123'", "TIMESTAMP '1958-01-01 13:18:03.123'") + // after epoch + .addRoundTrip("TIMESTAMP '2019-03-18 10:01:17.987'", "TIMESTAMP '2019-03-18 10:01:17.987'") + // time doubled in JVM zone + .addRoundTrip("TIMESTAMP '2018-10-28 01:33:17.456'", "TIMESTAMP '2018-10-28 01:33:17.456'") + // time double in Vilnius + .addRoundTrip("TIMESTAMP '2018-10-28 03:33:33.333'", "TIMESTAMP '2018-10-28 03:33:33.333'") + // epoch + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.000'", "TIMESTAMP '1970-01-01 00:00:00.000'") + // time gap in JVM zone + .addRoundTrip("TIMESTAMP '1970-01-01 00:13:42.000'", "TIMESTAMP '1970-01-01 00:13:42.000'") + .addRoundTrip("TIMESTAMP '2018-04-01 02:13:55.123'", "TIMESTAMP '2018-04-01 02:13:55.123'") + // time gap in Vilnius + .addRoundTrip("TIMESTAMP '2018-03-25 03:17:17.000'", "TIMESTAMP '2018-03-25 03:17:17.000'") + // time gap in Kathmandu + .addRoundTrip("TIMESTAMP '1986-01-01 00:13:07.000'", "TIMESTAMP '1986-01-01 00:13:07.000'") + + // same as above but with higher precision + .addRoundTrip("TIMESTAMP '1958-01-01 13:18:03.1230000'", "TIMESTAMP '1958-01-01 13:18:03.1230000'") + .addRoundTrip("TIMESTAMP '2019-03-18 10:01:17.9870000'", "TIMESTAMP '2019-03-18 10:01:17.9870000'") + .addRoundTrip("TIMESTAMP '2018-10-28 01:33:17.4560000'", "TIMESTAMP '2018-10-28 01:33:17.4560000'") + .addRoundTrip("TIMESTAMP '2018-10-28 03:33:33.3330000'", "TIMESTAMP '2018-10-28 03:33:33.3330000'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.0000000'", "TIMESTAMP '1970-01-01 00:00:00.0000000'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:13:42.0000000'", "TIMESTAMP '1970-01-01 00:13:42.0000000'") + .addRoundTrip("TIMESTAMP '2018-04-01 02:13:55.1230000'", "TIMESTAMP '2018-04-01 02:13:55.1230000'") + .addRoundTrip("TIMESTAMP '2018-03-25 03:17:17.0000000'", "TIMESTAMP '2018-03-25 03:17:17.0000000'") + .addRoundTrip("TIMESTAMP '1986-01-01 00:13:07.0000000'", "TIMESTAMP '1986-01-01 00:13:07.0000000'") + + // test arbitrary time for all supported precisions + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00'", "TIMESTAMP '1970-01-01 00:00:00'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1'", "TIMESTAMP '1970-01-01 00:00:00.1'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12'", "TIMESTAMP '1970-01-01 00:00:00.12'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123'", "TIMESTAMP '1970-01-01 00:00:00.123'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234'", "TIMESTAMP '1970-01-01 00:00:00.1234'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345'", "TIMESTAMP '1970-01-01 00:00:00.12345'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456'", "TIMESTAMP '1970-01-01 00:00:00.123456'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234567'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345678'", "TIMESTAMP '1970-01-01 00:00:00.12345678'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456789'", "TIMESTAMP '1970-01-01 00:00:00.123456789'") + // rounds + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.987987987111'", "TIMESTAMP '1970-01-01 00:00:00.987987987'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.987987987999'", "TIMESTAMP '1970-01-01 00:00:00.987987988'") + + // before epoch with second fraction + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.1230000'", "TIMESTAMP '1969-12-31 23:59:59.1230000'") + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.1234567'", "TIMESTAMP '1969-12-31 23:59:59.1234567'") + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.123456789'", "TIMESTAMP '1969-12-31 23:59:59.123456789'") + + // precision 0 ends up as precision 0 + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.9'", "TIMESTAMP '1970-01-01 00:00:00.9'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123000'", "TIMESTAMP '1970-01-01 00:00:00.123000'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.999'", "TIMESTAMP '1970-01-01 00:00:00.999'") + // max supported precision + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456000'", "TIMESTAMP '1970-01-01 00:00:00.123456000'") + + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.1'", "TIMESTAMP '2020-09-27 12:34:56.1'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.9'", "TIMESTAMP '2020-09-27 12:34:56.9'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.123'", "TIMESTAMP '2020-09-27 12:34:56.123'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.123000'", "TIMESTAMP '2020-09-27 12:34:56.123000'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.999'", "TIMESTAMP '2020-09-27 12:34:56.999'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.1234567'", "TIMESTAMP '2020-09-27 12:34:56.1234567'") + // max supported precision + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.123456789'", "TIMESTAMP '2020-09-27 12:34:56.123456789'") + + // max precision + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.111222333444'", "TIMESTAMP '1970-01-01 00:00:00.111222333'") + + // round up to next second + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.9999999995'", "TIMESTAMP '1970-01-01 00:00:01.000000000'") + + // round up to next day + .addRoundTrip("TIMESTAMP '1970-01-01 23:59:59.9999999995'", "TIMESTAMP '1970-01-02 00:00:00.000000000'") + + // negative epoch + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.9999999995'", "TIMESTAMP '1970-01-01 00:00:00.000000000'") + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.999999999499'", "TIMESTAMP '1969-12-31 23:59:59.999999999'") + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.9999999994'", "TIMESTAMP '1969-12-31 23:59:59.999999999'"); + + Session session = Session.builder(getSession()) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId())) + .build(); + tests.execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")); + tests.execute(getQueryRunner(), session, trinoCreateAsSelect("test_timestamp")); + tests.execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp")); + tests.execute(getQueryRunner(), session, trinoCreateAndInsert("test_timestamp")); + } + + @Test + public void testTimestampAllPrecisionsOnOracle() + { + SqlDataTypeTest.create() + .addRoundTrip("TIMESTAMP(0)", "TIMESTAMP '1970-01-01 00:00:00'", createTimestampType(0), "TIMESTAMP '1970-01-01 00:00:00'") + .addRoundTrip("TIMESTAMP(1)", "TIMESTAMP '1970-01-01 00:00:00.1'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:00.1'") + .addRoundTrip("TIMESTAMP(1)", "TIMESTAMP '1970-01-01 00:00:00.9'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:00.9'") + .addRoundTrip("TIMESTAMP(3)", "TIMESTAMP '1970-01-01 00:00:00.123'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:00.123'") + .addRoundTrip("TIMESTAMP(6)", "TIMESTAMP '1970-01-01 00:00:00.123000'", createTimestampType(6), "TIMESTAMP '1970-01-01 00:00:00.123000'") + .addRoundTrip("TIMESTAMP(3)", "TIMESTAMP '1970-01-01 00:00:00.999'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:00.999'") + .addRoundTrip("TIMESTAMP(7)", "TIMESTAMP '1970-01-01 00:00:00.1234567'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.123456789'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.123456789'") + .addRoundTrip("TIMESTAMP(1)", "TIMESTAMP '2020-09-27 12:34:56.1'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.1'") + .addRoundTrip("TIMESTAMP(1)", "TIMESTAMP '2020-09-27 12:34:56.9'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.9'") + .addRoundTrip("TIMESTAMP(3)", "TIMESTAMP '2020-09-27 12:34:56.123'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.123'") + .addRoundTrip("TIMESTAMP(6)", "TIMESTAMP '2020-09-27 12:34:56.123000'", createTimestampType(6), "TIMESTAMP '2020-09-27 12:34:56.123000'") + .addRoundTrip("TIMESTAMP(3)", "TIMESTAMP '2020-09-27 12:34:56.999'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.999'") + .addRoundTrip("TIMESTAMP(7)", "TIMESTAMP '2020-09-27 12:34:56.1234567'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1234567'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2020-09-27 12:34:56.123456789'", createTimestampType(9), "TIMESTAMP '2020-09-27 12:34:56.123456789'") + + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.000000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.1'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.100000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.9'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.900000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.123'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.123000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.123000'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.123000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.999'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.999000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.1234567'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.123456700'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '1970-01-01 00:00:00.123456789'", createTimestampType(9), "TIMESTAMP '1970-01-01 00:00:00.123456789'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2020-09-27 12:34:56.1'", createTimestampType(9), "TIMESTAMP '2020-09-27 12:34:56.100000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2020-09-27 12:34:56.9'", createTimestampType(9), "TIMESTAMP '2020-09-27 12:34:56.900000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2020-09-27 12:34:56.123'", createTimestampType(9), "TIMESTAMP '2020-09-27 12:34:56.123000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2020-09-27 12:34:56.123000'", createTimestampType(9), "TIMESTAMP '2020-09-27 12:34:56.123000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2020-09-27 12:34:56.999'", createTimestampType(9), "TIMESTAMP '2020-09-27 12:34:56.999000000'") + .addRoundTrip("TIMESTAMP(9)", "TIMESTAMP '2020-09-27 12:34:56.1234567'", createTimestampType(9), "TIMESTAMP '2020-09-27 12:34:56.123456700'") + + .execute(getQueryRunner(), oracleCreateAndInsert("test_ts_oracle")); + } + @Test public void testJulianGregorianTimestamp() { diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java index bbc8cf194da1..3b48c3a4889a 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java @@ -33,10 +33,12 @@ import static io.trino.spi.connector.ConnectorMetadata.MODIFYING_ROWS_MESSAGE; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static java.util.Locale.ENGLISH; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; public abstract class BaseOracleConnectorTest @@ -97,7 +99,6 @@ protected Optional filterDataMappingSmokeTestData(DataMapp } if (typeName.equals("time") || typeName.equals("time(6)") || - typeName.equals("timestamp(6)") || typeName.equals("timestamp(6) with time zone")) { return Optional.of(dataMappingTestSetup.asUnsupported()); } @@ -194,6 +195,18 @@ public void testShowCreateTable() ")"); } + @Test + public void testTimestampOutOfPrecisionRounded() + { + String tableName = "test_timestamp_" + randomNameSuffix(); + + assertUpdate("CREATE TABLE " + tableName + " (t timestamp(12))"); + + assertEquals(getColumnType(tableName, "t"), "timestamp(9)"); + + assertUpdate("DROP TABLE " + tableName); + } + @Override public void testCharVarcharComparison() { diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleClient.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleClient.java index 33953823ea7c..9685fa756e2d 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleClient.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleClient.java @@ -41,7 +41,9 @@ 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_MICROS; import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; import static io.trino.spi.type.TimestampType.TIMESTAMP_SECONDS; import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; import static io.trino.spi.type.TinyintType.TINYINT; @@ -112,7 +114,9 @@ public Object[][] writeMappingsProvider() {createUnboundedVarcharType(), "?", Types.VARCHAR}, {createVarcharType(123), "?", Types.VARCHAR}, {TIMESTAMP_SECONDS, "TO_DATE(?, 'SYYYY-MM-DD HH24:MI:SS')", Types.VARCHAR}, - {TIMESTAMP_MILLIS, "TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS.FF')", Types.VARCHAR}, + {TIMESTAMP_MILLIS, "TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS.FF3')", Types.VARCHAR}, + {TIMESTAMP_MICROS, "TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS.FF6')", Types.VARCHAR}, + {TIMESTAMP_NANOS, "TO_TIMESTAMP(?, 'SYYYY-MM-DD HH24:MI:SS.FF9')", Types.VARCHAR}, {TIMESTAMP_TZ_MILLIS, "?", OracleTypes.TIMESTAMPTZ}, {DATE, "TO_DATE(?, 'SYYYY-MM-DD')", Types.VARCHAR}, };