From e47f908d3ac06578266c586cff24d682512f1db3 Mon Sep 17 00:00:00 2001 From: regadas Date: Wed, 7 Jun 2023 16:30:46 -0400 Subject: [PATCH] Support writing timestamp with timezone in BigQuery --- .../trino/plugin/bigquery/BigQueryType.java | 7 +- .../plugin/bigquery/BigQueryTypeUtils.java | 9 +- .../bigquery/BaseBigQueryConnectorTest.java | 1 - .../bigquery/BaseBigQueryTypeMapping.java | 154 ++++++++++++++---- .../plugin/bigquery/TestBigQueryType.java | 6 +- 5 files changed, 139 insertions(+), 38 deletions(-) diff --git a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryType.java b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryType.java index 3f831329201f6..3c9c1b0d56aa9 100644 --- a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryType.java +++ b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryType.java @@ -94,7 +94,7 @@ private BigQueryType() {} 1, // 9 digits after the dot }; private static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("''HH:mm:ss.SSSSSS''"); - private static final DateTimeFormatter DATETIME_FORMATTER = DateTimeFormatter.ofPattern("''yyyy-MM-dd HH:mm:ss.SSSSSS''"); + private static final DateTimeFormatter DATETIME_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss.SSSSSS").withZone(UTC); private static RowType.Field toRawTypeField(String name, Field field) { @@ -158,7 +158,6 @@ public static String timeToStringConverter(Object value) return TIME_FORMATTER.format(toZonedDateTime(epochSeconds, nanoAdjustment, UTC)); } - @VisibleForTesting public static String timestampToStringConverter(Object value) { LongTimestampWithTimeZone timestamp = (LongTimestampWithTimeZone) value; @@ -289,7 +288,7 @@ public static Optional convertToString(Type type, StandardSQLTypeName bi case DATE: return Optional.of(dateToStringConverter(value)); case DATETIME: - return Optional.of(datetimeToStringConverter(value)); + return Optional.of(datetimeToStringConverter(value)).map("'%s'"::formatted); case FLOAT64: return Optional.of(floatToStringConverter(value)); case INT64: @@ -312,7 +311,7 @@ public static Optional convertToString(Type type, StandardSQLTypeName bi case TIME: return Optional.of(timeToStringConverter(value)); case TIMESTAMP: - return Optional.of(timestampToStringConverter(value)); + return Optional.of(timestampToStringConverter(value)).map("'%s'"::formatted); default: throw new IllegalArgumentException("Unsupported type: " + bigqueryType); } diff --git a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryTypeUtils.java b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryTypeUtils.java index d73b68aa81d5a..216dab6f8e686 100644 --- a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryTypeUtils.java +++ b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryTypeUtils.java @@ -18,6 +18,7 @@ import io.trino.spi.block.Block; import io.trino.spi.type.ArrayType; import io.trino.spi.type.DecimalType; +import io.trino.spi.type.LongTimestampWithTimeZone; import io.trino.spi.type.RowType; import io.trino.spi.type.Type; import io.trino.spi.type.VarcharType; @@ -31,6 +32,7 @@ import java.util.List; import java.util.Map; +import static io.trino.plugin.bigquery.BigQueryType.timestampToStringConverter; import static io.trino.plugin.bigquery.BigQueryType.toZonedDateTime; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; @@ -42,6 +44,7 @@ import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.SmallintType.SMALLINT; 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_SECOND; import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MICROSECOND; import static io.trino.spi.type.TinyintType.TINYINT; @@ -65,7 +68,7 @@ public static Object readNativeValue(Type type, Block block, int position) return null; } - // TODO https://github.com/trinodb/trino/issues/13741 Add support for time, timestamp with time zone, geography, map type + // TODO https://github.com/trinodb/trino/issues/13741 Add support for time, geography, map type if (type.equals(BOOLEAN)) { return BOOLEAN.getBoolean(block, position); } @@ -103,6 +106,10 @@ public static Object readNativeValue(Type type, Block block, int position) int nanoAdjustment = floorMod(epochMicros, MICROSECONDS_PER_SECOND) * NANOSECONDS_PER_MICROSECOND; return DATETIME_FORMATTER.format(toZonedDateTime(epochSeconds, nanoAdjustment, UTC)); } + if (type.equals(TIMESTAMP_TZ_MICROS)) { + LongTimestampWithTimeZone timestamp = (LongTimestampWithTimeZone) TIMESTAMP_TZ_MICROS.getObject(block, position); + return timestampToStringConverter(timestamp); + } if (type instanceof ArrayType arrayType) { Block arrayBlock = block.getObject(position, Block.class); ImmutableList.Builder list = ImmutableList.builderWithExpectedSize(arrayBlock.getPositionCount()); diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java index b8a0da7eb2d67..d4e90a1f36a34 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java @@ -265,7 +265,6 @@ protected Optional filterDataMappingSmokeTestData(DataMapp case "timestamp": case "timestamp(3)": case "timestamp(3) with time zone": - case "timestamp(6) with time zone": return Optional.of(dataMappingTestSetup.asUnsupported()); } return Optional.of(dataMappingTestSetup); diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java index 039aa2097bb35..ccb1cd23e6677 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryTypeMapping.java @@ -18,7 +18,9 @@ import io.trino.spi.type.ArrayType; import io.trino.spi.type.RowType; import io.trino.spi.type.RowType.Field; +import io.trino.spi.type.TimeZoneKey; import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.TestingSession; import io.trino.testing.datatype.CreateAndInsertDataSetup; import io.trino.testing.datatype.CreateAndTrinoInsertDataSetup; import io.trino.testing.datatype.CreateAsSelectDataSetup; @@ -31,6 +33,7 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.time.ZoneId; import java.util.Optional; import static io.trino.spi.type.BigintType.BIGINT; @@ -45,6 +48,7 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; +import static java.time.ZoneOffset.UTC; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** @@ -54,6 +58,9 @@ public abstract class BaseBigQueryTypeMapping extends AbstractTestQueryFramework { private BigQueryQueryRunner.BigQuerySqlExecutor bigQuerySqlExecutor; + private final ZoneId jvmZone = ZoneId.systemDefault(); + private final ZoneId vilnius = ZoneId.of("Europe/Vilnius"); + private final ZoneId kathmandu = ZoneId.of("Asia/Kathmandu"); @BeforeClass(alwaysRun = true) public void initBigQueryExecutor() @@ -526,44 +533,129 @@ public void testTime() .execute(getQueryRunner(), bigqueryViewCreateAndInsert("test.time")); } - @Test - public void testTimestampWithTimeZone() + @Test(dataProvider = "sessionZonesDataProvider") + public void testTimestampWithTimeZone(ZoneId zoneId) { - SqlDataTypeTest.create() + Session session = Session.builder(getSession()) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(zoneId.getId())) + .build(); + + testTimestampWithTimeZone("TIMESTAMP(6) WITH TIME ZONE") + .execute(getQueryRunner(), trinoCreateAsSelect("test.timestamp_tz")) + .execute(getQueryRunner(), trinoCreateAsSelect(session, "test.timestamp_tz")) + .execute(getQueryRunner(), session, trinoCreateAsSelect("test.timestamp_tz")); + + testTimestampWithTimeZone("TIMESTAMP") + .execute(getQueryRunner(), bigqueryCreateAndInsert("test.timestamp_tz")); + } + + private SqlDataTypeTest testTimestampWithTimeZone(String inputType) + { + return SqlDataTypeTest.create() // min value in BigQuery - .addRoundTrip("TIMESTAMP", "TIMESTAMP '0001-01-01 00:00:00.000000 UTC'", + .addRoundTrip(inputType, "TIMESTAMP '0001-01-01 00:00:00.000 UTC'", TIMESTAMP_TZ_MICROS, "TIMESTAMP '0001-01-01 00:00:00.000000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1970-01-01 00:00:00.000000 UTC'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:00.000000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1970-01-01 00:00:00.000000 Asia/Kathmandu'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '1969-12-31 18:30:00.000000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1970-01-01 00:00:00.000000+02:17'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '1969-12-31 21:43:00.000000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1970-01-01 00:00:00.000000-07:31'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 07:31:00.000000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1958-01-01 13:18:03.123456 UTC'", + // before epoch + .addRoundTrip(inputType, "TIMESTAMP '1958-01-01 13:18:03.123 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1958-01-01 13:18:03.123000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1958-01-01 13:18:03.123456 UTC'", TIMESTAMP_TZ_MICROS, "TIMESTAMP '1958-01-01 13:18:03.123456 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1958-01-01 13:18:03.123000 Asia/Kathmandu'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '1958-01-01 07:48:03.123000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1958-01-01 13:18:03.123000+02:17'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '1958-01-01 11:01:03.123000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '1958-01-01 13:18:03.123000-07:31'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '1958-01-01 20:49:03.123000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '2019-03-18 10:01:17.987654 UTC'", + .addRoundTrip(inputType, "TIMESTAMP '1958-01-01 13:18:03.123000 Asia/Kathmandu'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1958-01-01 07:48:03.123000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1969-12-31 23:59:59.999995 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1969-12-31 23:59:59.999995 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1969-12-31 23:59:59.999949 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1969-12-31 23:59:59.999949 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1969-12-31 23:59:59.999994 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1969-12-31 23:59:59.999994 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:00.000000 Asia/Kathmandu'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1969-12-31 18:30:00.000000 UTC'") + // epoch + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:00.000 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:00.000000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:00.000000 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:00.000000 UTC'") + // after epoch + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:01 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:01.000000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:01.1 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:01.100000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:01.12 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:01.120000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:01.123 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:01.123000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:01.1234 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:01.123400 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:01.12345 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:01.123450 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:00:01.123456 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:00:01.123456 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:13:42.000 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:13:42.000000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1970-01-01 00:13:42.123456 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1970-01-01 00:13:42.123456 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1986-01-01 00:13:07.000 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1986-01-01 00:13:07.000000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '1986-01-01 00:13:07.456789 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '1986-01-01 00:13:07.456789 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-03-25 03:17:17.000 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-03-25 03:17:17.000000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-03-25 03:17:17.456789 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-03-25 03:17:17.456789 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-04-01 02:13:55.123 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-04-01 02:13:55.123000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-04-01 02:13:55.123456 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-04-01 02:13:55.123456 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-10-28 01:33:17.456 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-10-28 01:33:17.456000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-10-28 01:33:17.123456 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-10-28 01:33:17.123456 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-10-28 03:33:33.333 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-10-28 03:33:33.333000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2018-10-28 03:33:33.333333 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2018-10-28 03:33:33.333333 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2019-03-18 10:01:17.987 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '2019-03-18 10:01:17.987000 UTC'") + .addRoundTrip(inputType, "TIMESTAMP '2019-03-18 10:01:17.987654 UTC'", TIMESTAMP_TZ_MICROS, "TIMESTAMP '2019-03-18 10:01:17.987654 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '2019-03-18 10:01:17.987000 Asia/Kathmandu'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '2019-03-18 04:16:17.987000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '2019-03-18 10:01:17.987000+02:17'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '2019-03-18 07:44:17.987000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '2019-03-18 10:01:17.987000-07:31'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '2019-03-18 17:32:17.987000 UTC'") - .addRoundTrip("TIMESTAMP", "TIMESTAMP '2021-09-07 23:59:59.999999-00:00'", + .addRoundTrip(inputType, "TIMESTAMP '2021-09-07 23:59:59.999999 UTC'", TIMESTAMP_TZ_MICROS, "TIMESTAMP '2021-09-07 23:59:59.999999 UTC'") // max value in BigQuery - .addRoundTrip("TIMESTAMP", "TIMESTAMP '9999-12-31 23:59:59.999999-00:00'", - TIMESTAMP_TZ_MICROS, "TIMESTAMP '9999-12-31 23:59:59.999999 UTC'") - .execute(getQueryRunner(), bigqueryCreateAndInsert("test.timestamp_tz")); - // TODO (https://github.com/trinodb/trino/pull/12210) Add support for timestamp with time zone type in views + .addRoundTrip(inputType, "TIMESTAMP '9999-12-31 23:59:59.999999 UTC'", + TIMESTAMP_TZ_MICROS, "TIMESTAMP '9999-12-31 23:59:59.999999 UTC'"); + } + + @DataProvider + public Object[][] sessionZonesDataProvider() + { + return new Object[][] { + {UTC}, + {jvmZone}, + // using two non-JVM zones so that we don't need to worry what BigQuery system zone is + {vilnius}, + {kathmandu}, + {TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId()}, + }; + } + + @Test + public void testUnsupportedTimestampWithTimeZone() + { + try (TestTable table = new TestTable(getBigQuerySqlExecutor(), "test.unsupported_tz", "(col timestamp)")) { + assertQueryFails("INSERT INTO " + table.getName() + " VALUES (timestamp '-2021-09-07 23:59:59.999999 UTC')", "Failed to insert rows.*"); + assertQueryFails("INSERT INTO " + table.getName() + " VALUES (timestamp '-0001-01-01 00:00:00.000000 UTC')", "Failed to insert rows.*"); + assertQueryFails("INSERT INTO " + table.getName() + " VALUES (timestamp '0000-12-31 23:59:59.999999 UTC')", "Failed to insert rows.*"); + assertQueryFails("INSERT INTO " + table.getName() + " VALUES (timestamp '10000-01-01 00:00:00.000000 UTC')", "Failed to insert rows.*"); + + assertThatThrownBy(() -> getBigQuerySqlExecutor().execute("INSERT INTO " + table.getName() + " VALUES (timestamp '-2021-09-07 23:59:59.999999 UTC')")) + .hasMessageContaining("Invalid TIMESTAMP literal"); + assertThatThrownBy(() -> getBigQuerySqlExecutor().execute("INSERT INTO " + table.getName() + " VALUES (timestamp '-0001-01-01 00:00:00.000000 UTC')")) + .hasMessageContaining("Invalid TIMESTAMP literal"); + assertThatThrownBy(() -> getBigQuerySqlExecutor().execute("INSERT INTO " + table.getName() + " VALUES (timestamp '0000-12-31 23:59:59.999999 UTC')")) + .hasMessageContaining("Invalid TIMESTAMP literal"); + assertThatThrownBy(() -> getBigQuerySqlExecutor().execute("INSERT INTO " + table.getName() + " VALUES (timestamp '10000-01-01 00:00:00.000000 UTC')")) + .hasMessageContaining("Invalid TIMESTAMP literal"); + } } @Test diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryType.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryType.java index fcfa1e7db8706..82108607415cd 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryType.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/TestBigQueryType.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.bigquery; +import io.trino.spi.type.TimeZoneKey; import org.testng.annotations.Test; import static io.airlift.slice.Slices.utf8Slice; @@ -42,7 +43,10 @@ public void testTimestampToStringConverter() { assertThat(BigQueryType.timestampToStringConverter( fromEpochSecondsAndFraction(1585658096, 123_456_000_000L, UTC_KEY))) - .isEqualTo("'2020-03-31 12:34:56.123456'"); + .isEqualTo("2020-03-31 12:34:56.123456"); + assertThat(BigQueryType.timestampToStringConverter( + fromEpochSecondsAndFraction(1585658096, 123_456_000_000L, TimeZoneKey.getTimeZoneKey("Asia/Kathmandu")))) + .isEqualTo("2020-03-31 12:34:56.123456"); } @Test