diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 1252b16f93b4..b4409d0735e4 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -102,6 +102,7 @@ import io.trino.spi.statistics.TableStatisticsMetadata; import io.trino.spi.type.LongTimestamp; import io.trino.spi.type.LongTimestampWithTimeZone; +import io.trino.spi.type.TimeType; import io.trino.spi.type.TimestampType; import io.trino.spi.type.TimestampWithTimeZoneType; import io.trino.spi.type.TypeManager; @@ -266,6 +267,9 @@ import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.DateTimeEncoding.unpackMillisUtc; import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.TimeType.TIME_MICROS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS; +import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MICROS; import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; import static io.trino.spi.type.UuidType.UUID; import static java.lang.Math.floorDiv; @@ -815,6 +819,21 @@ public Optional getNewTableLayout(ConnectorSession session return getWriteLayout(schema, partitionSpec, false); } + @Override + public Optional getSupportedType(ConnectorSession session, io.trino.spi.type.Type type) + { + if (type instanceof TimestampWithTimeZoneType) { + return Optional.of(TIMESTAMP_TZ_MICROS); + } + if (type instanceof TimestampType) { + return Optional.of(TIMESTAMP_MICROS); + } + if (type instanceof TimeType) { + return Optional.of(TIME_MICROS); + } + return Optional.empty(); + } + @Override public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, Optional layout, RetryMode retryMode) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index a24721237690..771aed4c66a9 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -128,6 +128,7 @@ import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.assertions.PlanMatchPattern.node; +import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; import static io.trino.testing.TestingConnectorSession.SESSION; @@ -4685,7 +4686,7 @@ public void testSplitPruningForFilterOnNonPartitionColumn(DataMappingTestSetup t verifySplitCount("SELECT row_id FROM " + tableName + " WHERE col > " + sampleValue, (format == ORC && testSetup.getTrinoTypeName().contains("timestamp") ? 2 : expectedSplitCount)); verifySplitCount("SELECT row_id FROM " + tableName + " WHERE col < " + highValue, - (format == ORC && testSetup.getTrinoTypeName().contains("timestamp") ? 2 : expectedSplitCount)); + (format == ORC && testSetup.getTrinoTypeName().contains("timestamp(6)") ? 2 : expectedSplitCount)); } } @@ -4816,14 +4817,6 @@ protected Optional filterDataMappingSmokeTestData(DataMapp // These types are not supported by Iceberg return Optional.of(dataMappingTestSetup.asUnsupported()); } - - // According to Iceberg specification all time and timestamp values are stored with microsecond precision. - if (typeName.equals("time") || - typeName.equals("timestamp") || - typeName.equals("timestamp(3) with time zone")) { - return Optional.of(dataMappingTestSetup.asUnsupported()); - } - return Optional.of(dataMappingTestSetup); } @@ -6940,9 +6933,154 @@ protected void verifyTableNameLengthFailurePermissible(Throwable e) assertThat(e).hasMessageMatching("Table name must be shorter than or equal to '128' characters but got .*"); } + @Test(dataProvider = "testTimestampPrecisionOnCreateTableAsSelect") + public void testTimestampPrecisionOnCreateTableAsSelect(TimestampPrecisionTestSetup setup) + { + try (TestTable testTable = new TestTable( + getQueryRunner()::execute, + "test_coercion_show_create_table", + format("AS SELECT %s a", setup.sourceValueLiteral))) { + assertEquals(getColumnType(testTable.getName(), "a"), setup.newColumnType); + assertQuery( + format("SELECT * FROM %s", testTable.getName()), + format("VALUES (%s)", setup.newValueLiteral)); + } + } + + @Test(dataProvider = "testTimestampPrecisionOnCreateTableAsSelect") + public void testTimestampPrecisionOnCreateTableAsSelectWithNoData(TimestampPrecisionTestSetup setup) + { + try (TestTable testTable = new TestTable( + getQueryRunner()::execute, + "test_coercion_show_create_table", + format("AS SELECT %s a WITH NO DATA", setup.sourceValueLiteral))) { + assertEquals(getColumnType(testTable.getName(), "a"), setup.newColumnType); + } + } + + @DataProvider(name = "testTimestampPrecisionOnCreateTableAsSelect") + public Object[][] timestampPrecisionOnCreateTableAsSelectProvider() + { + return timestampPrecisionOnCreateTableAsSelectData().stream() + .map(this::filterTimestampPrecisionOnCreateTableAsSelectProvider) + .flatMap(Optional::stream) + .collect(toDataProvider()); + } + + protected Optional filterTimestampPrecisionOnCreateTableAsSelectProvider(TimestampPrecisionTestSetup setup) + { + return Optional.of(setup); + } + + private List timestampPrecisionOnCreateTableAsSelectData() + { + return ImmutableList.builder() + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.000000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.9'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.900000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.56'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.560000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.123'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.123000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.4896'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.489600'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.89356'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.893560'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.123000'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.123000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.999'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.999000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.123456'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.123456'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '2020-09-27 12:34:56.1'", "timestamp(6)", "TIMESTAMP '2020-09-27 12:34:56.100000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '2020-09-27 12:34:56.9'", "timestamp(6)", "TIMESTAMP '2020-09-27 12:34:56.900000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '2020-09-27 12:34:56.123'", "timestamp(6)", "TIMESTAMP '2020-09-27 12:34:56.123000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '2020-09-27 12:34:56.123000'", "timestamp(6)", "TIMESTAMP '2020-09-27 12:34:56.123000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '2020-09-27 12:34:56.999'", "timestamp(6)", "TIMESTAMP '2020-09-27 12:34:56.999000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '2020-09-27 12:34:56.123456'", "timestamp(6)", "TIMESTAMP '2020-09-27 12:34:56.123456'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.1234561'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.123456'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.123456499'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.123456'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.123456499999'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.123456'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.1234565'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.123457'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.111222333444'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.111222'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 00:00:00.9999995'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:01.000000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1970-01-01 23:59:59.9999995'", "timestamp(6)", "TIMESTAMP '1970-01-02 00:00:00.000000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1969-12-31 23:59:59.9999995'", "timestamp(6)", "TIMESTAMP '1970-01-01 00:00:00.000000'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1969-12-31 23:59:59.999999499999'", "timestamp(6)", "TIMESTAMP '1969-12-31 23:59:59.999999'")) + .add(new TimestampPrecisionTestSetup("TIMESTAMP '1969-12-31 23:59:59.9999994'", "timestamp(6)", "TIMESTAMP '1969-12-31 23:59:59.999999'")) + .build(); + } + + public record TimestampPrecisionTestSetup(String sourceValueLiteral, String newColumnType, String newValueLiteral) + { + public TimestampPrecisionTestSetup + { + requireNonNull(sourceValueLiteral, "sourceValueLiteral is null"); + requireNonNull(newColumnType, "newColumnType is null"); + requireNonNull(newValueLiteral, "newValueLiteral is null"); + } + + public TimestampPrecisionTestSetup withNewValueLiteral(String newValueLiteral) + { + return new TimestampPrecisionTestSetup(sourceValueLiteral, newColumnType, newValueLiteral); + } + } + + @Test(dataProvider = "testTimePrecisionOnCreateTableAsSelect") + public void testTimePrecisionOnCreateTableAsSelect(String inputType, String tableType, String tableValue) + { + try (TestTable testTable = new TestTable( + getQueryRunner()::execute, + "test_coercion_show_create_table", + format("AS SELECT %s a", inputType))) { + assertEquals(getColumnType(testTable.getName(), "a"), tableType); + assertQuery( + format("SELECT * FROM %s", testTable.getName()), + format("VALUES (%s)", tableValue)); + } + } + + @Test(dataProvider = "testTimePrecisionOnCreateTableAsSelect") + public void testTimePrecisionOnCreateTableAsSelectWithNoData(String inputType, String tableType, String ignored) + { + try (TestTable testTable = new TestTable( + getQueryRunner()::execute, + "test_coercion_show_create_table", + format("AS SELECT %s a WITH NO DATA", inputType))) { + assertEquals(getColumnType(testTable.getName(), "a"), tableType); + } + } + + @DataProvider(name = "testTimePrecisionOnCreateTableAsSelect") + public static Object[][] timePrecisionOnCreateTableAsSelectProvider() + { + return new Object[][] { + {"TIME '00:00:00'", "time(6)", "TIME '00:00:00.000000'"}, + {"TIME '00:00:00.9'", "time(6)", "TIME '00:00:00.900000'"}, + {"TIME '00:00:00.56'", "time(6)", "TIME '00:00:00.560000'"}, + {"TIME '00:00:00.123'", "time(6)", "TIME '00:00:00.123000'"}, + {"TIME '00:00:00.4896'", "time(6)", "TIME '00:00:00.489600'"}, + {"TIME '00:00:00.89356'", "time(6)", "TIME '00:00:00.893560'"}, + {"TIME '00:00:00.123000'", "time(6)", "TIME '00:00:00.123000'"}, + {"TIME '00:00:00.999'", "time(6)", "TIME '00:00:00.999000'"}, + {"TIME '00:00:00.123456'", "time(6)", "TIME '00:00:00.123456'"}, + {"TIME '12:34:56.1'", "time(6)", "TIME '12:34:56.100000'"}, + {"TIME '12:34:56.9'", "time(6)", "TIME '12:34:56.900000'"}, + {"TIME '12:34:56.123'", "time(6)", "TIME '12:34:56.123000'"}, + {"TIME '12:34:56.123000'", "time(6)", "TIME '12:34:56.123000'"}, + {"TIME '12:34:56.999'", "time(6)", "TIME '12:34:56.999000'"}, + {"TIME '12:34:56.123456'", "time(6)", "TIME '12:34:56.123456'"}, + {"TIME '00:00:00.1234561'", "time(6)", "TIME '00:00:00.123456'"}, + {"TIME '00:00:00.123456499'", "time(6)", "TIME '00:00:00.123456'"}, + {"TIME '00:00:00.123456499999'", "time(6)", "TIME '00:00:00.123456'"}, + {"TIME '00:00:00.1234565'", "time(6)", "TIME '00:00:00.123457'"}, + {"TIME '00:00:00.111222333444'", "time(6)", "TIME '00:00:00.111222'"}, + {"TIME '00:00:00.9999995'", "time(6)", "TIME '00:00:01.000000'"}, + {"TIME '23:59:59.9999995'", "time(6)", "TIME '00:00:00.000000'"}, + {"TIME '23:59:59.9999995'", "time(6)", "TIME '00:00:00.000000'"}, + {"TIME '23:59:59.999999499999'", "time(6)", "TIME '23:59:59.999999'"}, + {"TIME '23:59:59.9999994'", "time(6)", "TIME '23:59:59.999999'"}}; + } + @Override protected Optional filterSetColumnTypesDataProvider(SetColumnTypeSetup setup) { + if (setup.sourceColumnType().equals("timestamp(3) with time zone")) { + // The connector returns UTC instead of the given time zone + return Optional.of(setup.withNewValueLiteral("TIMESTAMP '2020-02-12 14:03:00.123000 +00:00'")); + } switch ("%s -> %s".formatted(setup.sourceColumnType(), setup.newColumnType())) { case "bigint -> integer": case "decimal(5,3) -> decimal(5,2)": diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java index 1d9cad76a8cf..65d9d82c00d3 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java @@ -26,6 +26,7 @@ import java.nio.file.Files; import java.util.List; import java.util.Map; +import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.io.Resources.getResource; @@ -163,4 +164,16 @@ public void testDropAmbiguousRowFieldCaseSensitivity() .hasMessageContaining("Error opening Iceberg split") .hasStackTraceContaining("Multiple entries with same key"); } + + @Override + protected Optional filterTimestampPrecisionOnCreateTableAsSelectProvider(TimestampPrecisionTestSetup setup) + { + if (setup.sourceValueLiteral().equals("TIMESTAMP '1969-12-31 23:59:59.999999499999'")) { + return Optional.of(setup.withNewValueLiteral("TIMESTAMP '1970-01-01 00:00:00.999999'")); + } + if (setup.sourceValueLiteral().equals("TIMESTAMP '1969-12-31 23:59:59.9999994'")) { + return Optional.of(setup.withNewValueLiteral("TIMESTAMP '1970-01-01 00:00:00.999999'")); + } + return Optional.of(setup); + } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java index 16633c9f091f..35b9c65bbcc7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java @@ -45,7 +45,9 @@ protected boolean supportsIcebergFileStatistics(String typeName) protected boolean supportsRowGroupStatistics(String typeName) { return !(typeName.equalsIgnoreCase("varbinary") || + typeName.equalsIgnoreCase("time") || typeName.equalsIgnoreCase("time(6)") || + typeName.equalsIgnoreCase("timestamp(3) with time zone") || typeName.equalsIgnoreCase("timestamp(6) with time zone")); }