diff --git a/presto-blackhole/src/test/java/com/facebook/presto/plugin/blackhole/TestBlackHoleSmoke.java b/presto-blackhole/src/test/java/com/facebook/presto/plugin/blackhole/TestBlackHoleSmoke.java index ab6e719a4381..ab4e119b1b8b 100644 --- a/presto-blackhole/src/test/java/com/facebook/presto/plugin/blackhole/TestBlackHoleSmoke.java +++ b/presto-blackhole/src/test/java/com/facebook/presto/plugin/blackhole/TestBlackHoleSmoke.java @@ -258,7 +258,7 @@ public void testSelectAllTypes() assertEquals(row.getField(6), 0.0); assertEquals(row.getField(7), false); assertEquals(row.getField(8), LocalDate.ofEpochDay(0)); - assertEquals(row.getField(9), LocalDateTime.of(1970, 1, 1, 0, 0, 0)); + assertEquals(row.getField(9), LocalDateTime.of(1969, 12, 31, 13, 0, 0)); // TODO #7122 should be 1970-01-01 00:00:00 assertEquals(row.getField(10), "****************".getBytes()); assertEquals(row.getField(11), new BigDecimal("0.00")); assertEquals(row.getField(12), new BigDecimal("00000000000000000000.0000000000")); diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java index 098492cee675..855ff8cae6db 100644 --- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java +++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestCassandraIntegrationSmokeTest.java @@ -20,14 +20,13 @@ import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest; import com.google.common.collect.ImmutableList; import io.airlift.units.Duration; -import org.joda.time.DateTime; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; import java.math.BigInteger; import java.nio.ByteBuffer; +import java.sql.Timestamp; import java.time.LocalDateTime; -import java.util.Date; import java.util.List; import static com.datastax.driver.core.utils.Bytes.toRawHexString; @@ -56,7 +55,6 @@ import static com.google.common.primitives.Ints.toByteArray; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.stream.Collectors.toList; -import static org.joda.time.DateTimeZone.UTC; import static org.testng.Assert.assertEquals; @Test(singleThreaded = true) @@ -66,9 +64,8 @@ public class TestCassandraIntegrationSmokeTest private static final String KEYSPACE = "smoke_test"; private static final Session SESSION = createCassandraSession(KEYSPACE); - private static final DateTime DATE_TIME_UTC = new DateTime(1970, 1, 1, 3, 4, 5, UTC); - private static final Date DATE_LOCAL = new Date(DATE_TIME_UTC.getMillis()); - private static final LocalDateTime TIMESTAMP_LOCAL = LocalDateTime.of(1970, 1, 1, 3, 4, 5); + private static final Timestamp DATE_TIME_LOCAL = Timestamp.valueOf(LocalDateTime.of(1970, 1, 1, 3, 4, 5, 0)); + private static final LocalDateTime TIMESTAMP_LOCAL = LocalDateTime.of(1969, 12, 31, 23, 4, 5); // TODO #7122 should match DATE_TIME_LOCAL private CassandraSession session; @@ -81,7 +78,7 @@ public TestCassandraIntegrationSmokeTest() public void setUp() { session = EmbeddedCassandra.getSession(); - createTestTables(session, KEYSPACE, DATE_LOCAL); + createTestTables(session, KEYSPACE, DATE_TIME_LOCAL); } @Test @@ -94,7 +91,7 @@ public void testPartitionKeyPredicate() " AND typeinteger = 7" + " AND typelong = 1007" + " AND typebytes = from_hex('" + toRawHexString(ByteBuffer.wrap(toByteArray(7))) + "')" + - " AND typetimestamp = TIMESTAMP '1970-01-01 03:04:05'" + + " AND typetimestamp = TIMESTAMP '1969-12-31 23:04:05'" + " AND typeansi = 'ansi 7'" + " AND typeboolean = false" + " AND typedecimal = 128.0" + @@ -225,17 +222,17 @@ public void testClusteringKeyPushdownInequality() assertEquals(execute(sql).getRowCount(), 4); sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2"; assertEquals(execute(sql).getRowCount(), 1); - sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three = timestamp '1970-01-01 03:04:05.020'"; + sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three = timestamp '1969-12-31 23:04:05.020'"; assertEquals(execute(sql).getRowCount(), 1); - sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three = timestamp '1970-01-01 03:04:05.010'"; + sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three = timestamp '1969-12-31 23:04:05.010'"; assertEquals(execute(sql).getRowCount(), 0); sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2)"; assertEquals(execute(sql).getRowCount(), 2); sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two > 1 AND clust_two < 3"; assertEquals(execute(sql).getRowCount(), 1); - sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three >= timestamp '1970-01-01 03:04:05.010' AND clust_three <= timestamp '1970-01-01 03:04:05.020'"; + sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two=2 AND clust_three >= timestamp '1969-12-31 23:04:05.010' AND clust_three <= timestamp '1969-12-31 23:04:05.020'"; assertEquals(execute(sql).getRowCount(), 1); - sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2) AND clust_three >= timestamp '1970-01-01 03:04:05.010' AND clust_three <= timestamp '1970-01-01 03:04:05.020'"; + sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2) AND clust_three >= timestamp '1969-12-31 23:04:05.010' AND clust_three <= timestamp '1969-12-31 23:04:05.020'"; assertEquals(execute(sql).getRowCount(), 2); sql = "SELECT * FROM " + TABLE_CLUSTERING_KEYS_INEQUALITY + " WHERE key='key_1' AND clust_one='clust_one' AND clust_two IN (1,2,3) AND clust_two < 2"; assertEquals(execute(sql).getRowCount(), 1); diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java index 73a630010629..6d764c95a9a9 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingSession.java @@ -27,6 +27,7 @@ import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.facebook.presto.spi.security.Identity; import com.facebook.presto.spi.transaction.IsolationLevel; +import com.facebook.presto.spi.type.TimeZoneKey; import com.facebook.presto.sql.SqlPath; import com.google.common.collect.ImmutableSet; @@ -34,7 +35,6 @@ import static com.facebook.presto.connector.ConnectorId.createInformationSchemaConnectorId; import static com.facebook.presto.connector.ConnectorId.createSystemTablesConnectorId; -import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY; import static java.util.Locale.ENGLISH; public final class TestingSession @@ -42,6 +42,15 @@ public final class TestingSession public static final String TESTING_CATALOG = "testing_catalog"; private static final QueryIdGenerator queryIdGenerator = new QueryIdGenerator(); + /* + * Pacific/Apia + * - has DST (e.g. January 2017) + * - had DST change at midnight (on Sunday, 26 September 2010, 00:00:00 clocks were turned forward 1 hour) + * - had offset change since 1970 (offset in January 1970: -11:00, offset in January 2017: +14:00, offset in June 2017: +13:00) + * - a whole day was skipped during policy change (on Friday, 30 December 2011, 00:00:00 clocks were turned forward 24 hours) + */ + public static final TimeZoneKey DEFAULT_TIME_ZONE_KEY = TimeZoneKey.getTimeZoneKey("Pacific/Apia"); + private TestingSession() {} public static SessionBuilder testSessionBuilder() @@ -58,7 +67,7 @@ public static SessionBuilder testSessionBuilder(SessionPropertyManager sessionPr .setCatalog("catalog") .setSchema("schema") .setPath(new SqlPath(Optional.of("path"))) - .setTimeZoneKey(UTC_KEY) + .setTimeZoneKey(DEFAULT_TIME_ZONE_KEY) .setLocale(ENGLISH) .setRemoteUserAddress("address") .setUserAgent("agent"); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctions.java index 28eba3ccd583..38469441e013 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctions.java @@ -89,8 +89,8 @@ public void testCurrentTimestamp() .setStartTime(new DateTime(2017, 3, 1, 14, 30, 0, 0, DATE_TIME_ZONE).getMillis()) .build(); try (FunctionAssertions localAssertion = new FunctionAssertions(localSession)) { - localAssertion.assertFunctionString("CURRENT_TIMESTAMP", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 Asia/Kabul"); - localAssertion.assertFunctionString("NOW()", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 Asia/Kabul"); + localAssertion.assertFunctionString("CURRENT_TIMESTAMP", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 " + DATE_TIME_ZONE.getID()); + localAssertion.assertFunctionString("NOW()", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 " + DATE_TIME_ZONE.getID()); } } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsBase.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsBase.java index f4b8dcbc6c93..aadf63eebf6d 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsBase.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsBase.java @@ -26,6 +26,7 @@ import com.facebook.presto.spi.type.TimestampType; import com.facebook.presto.spi.type.Type; import com.facebook.presto.testing.TestingConnectorSession; +import com.facebook.presto.testing.TestingSession; import com.facebook.presto.type.SqlIntervalDayTime; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -80,10 +81,10 @@ public abstract class TestDateTimeFunctionsBase extends AbstractTestFunctions { - protected static final TimeZoneKey TIME_ZONE_KEY = getTimeZoneKey("Asia/Kabul"); + protected static final TimeZoneKey TIME_ZONE_KEY = TestingSession.DEFAULT_TIME_ZONE_KEY; protected static final DateTimeZone DATE_TIME_ZONE = getDateTimeZone(TIME_ZONE_KEY); protected static final DateTimeZone UTC_TIME_ZONE = getDateTimeZone(UTC_KEY); - protected static final DateTimeZone DATE_TIME_ZONE_NUMERICAL = getDateTimeZone(getTimeZoneKey("+04:30")); + protected static final DateTimeZone DATE_TIME_ZONE_NUMERICAL = getDateTimeZone(getTimeZoneKey("-11:00")); protected static final TimeZoneKey KATHMANDU_ZONE_KEY = getTimeZoneKey("Asia/Kathmandu"); protected static final DateTimeZone KATHMANDU_ZONE = getDateTimeZone(KATHMANDU_ZONE_KEY); protected static final TimeZoneKey WEIRD_ZONE_KEY = getTimeZoneKey("+07:09"); @@ -102,7 +103,7 @@ public abstract class TestDateTimeFunctionsBase protected static final DateTime LEGACY_TIMESTAMP = new DateTime(2001, 8, 22, 3, 4, 5, 321, DATE_TIME_ZONE); protected static final DateTime TIMESTAMP_WITH_NUMERICAL_ZONE = new DateTime(2001, 8, 22, 3, 4, 5, 321, DATE_TIME_ZONE_NUMERICAL); protected static final String TIMESTAMP_LITERAL = "TIMESTAMP '2001-08-22 03:04:05.321'"; - protected static final String TIMESTAMP_ISO8601_STRING = "2001-08-22T03:04:05.321+04:30"; + protected static final String TIMESTAMP_ISO8601_STRING = "2001-08-22T03:04:05.321-11:00"; protected static final String TIMESTAMP_ISO8601_STRING_NO_TIME_ZONE = "2001-08-22T03:04:05.321"; protected static final DateTime WEIRD_TIMESTAMP = new DateTime(2001, 8, 22, 3, 4, 5, 321, WEIRD_ZONE); protected static final String WEIRD_TIMESTAMP_LITERAL = "TIMESTAMP '2001-08-22 03:04:05.321 +07:09'"; @@ -273,9 +274,9 @@ public void testPartFunctions() assertFunction("month(" + TIMESTAMP_LITERAL + ")", BIGINT, (long) TIMESTAMP.getMonthOfYear()); assertFunction("quarter(" + TIMESTAMP_LITERAL + ")", BIGINT, (long) TIMESTAMP.getMonthOfYear() / 4 + 1); assertFunction("year(" + TIMESTAMP_LITERAL + ")", BIGINT, (long) TIMESTAMP.getYear()); - assertFunction("timezone_hour(" + TIMESTAMP_LITERAL + ")", BIGINT, 4L); - assertFunction("timezone_hour(localtimestamp)", BIGINT, 4L); - assertFunction("timezone_hour(current_timestamp)", BIGINT, 4L); + assertFunction("timezone_hour(" + TIMESTAMP_LITERAL + ")", BIGINT, -11L); + assertFunction("timezone_hour(localtimestamp)", BIGINT, 14L); + assertFunction("timezone_hour(current_timestamp)", BIGINT, 14L); assertFunction("second(" + WEIRD_TIMESTAMP_LITERAL + ")", BIGINT, (long) WEIRD_TIMESTAMP.getSecondOfMinute()); assertFunction("minute(" + WEIRD_TIMESTAMP_LITERAL + ")", BIGINT, (long) WEIRD_TIMESTAMP.getMinuteOfHour()); @@ -551,6 +552,7 @@ public void testAddFieldToTimestamp() @Test public void testAddFieldToDate() { + assertFunction("date_add('day', 0, " + DATE_LITERAL + ")", DateType.DATE, toDate(DATE)); assertFunction("date_add('day', 3, " + DATE_LITERAL + ")", DateType.DATE, toDate(DATE.plusDays(3))); assertFunction("date_add('week', 3, " + DATE_LITERAL + ")", DateType.DATE, toDate(DATE.plusWeeks(3))); assertFunction("date_add('month', 3, " + DATE_LITERAL + ")", DateType.DATE, toDate(DATE.plusMonths(3))); @@ -561,6 +563,7 @@ public void testAddFieldToDate() @Test public void testAddFieldToTime() { + assertFunction("date_add('millisecond', 0, " + TIME_LITERAL + ")", TimeType.TIME, toTime(TIME)); assertFunction("date_add('millisecond', 3, " + TIME_LITERAL + ")", TimeType.TIME, toTime(TIME.plusMillis(3))); assertFunction("date_add('second', 3, " + TIME_LITERAL + ")", TimeType.TIME, toTime(TIME.plusSeconds(3))); assertFunction("date_add('minute', 3, " + TIME_LITERAL + ")", TimeType.TIME, toTime(TIME.plusMinutes(3))); @@ -1005,10 +1008,11 @@ public void testTimeWithTimeZoneAtTimeZone() TIME_WITH_TIME_ZONE, new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 4, 15, 0, 0, UTC_TIME_ZONE).getMillis(), TimeZoneKey.UTC_KEY)); - // Noop on Asia/Kabul + // Noop when time zone doesn't change + TimeZoneKey kabul = TimeZoneKey.getTimeZoneKey("Asia/Kabul"); assertFunction("at_timezone(TIME '10:00 Asia/Kabul', 'Asia/Kabul')", TIME_WITH_TIME_ZONE, - new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 10, 0, 0, 0, DATE_TIME_ZONE).getMillis(), TIME_ZONE_KEY)); + new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 10, 0, 0, 0, getDateTimeZone(kabul)).getMillis(), kabul)); // This test checks if the TZ offset isn't calculated on other fixed point in time by checking if // session started in 1980 would get historical Asia/Kathmandu offset. diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsLegacy.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsLegacy.java index 6027d7f60f4d..77cfd0699f6f 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsLegacy.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestDateTimeFunctionsLegacy.java @@ -42,7 +42,7 @@ public void testToIso8601ForTimestampWithoutTimeZone() @Test public void testFormatDateCanImplicitlyAddTimeZoneToTimestampLiteral() { - assertFunction("format_datetime(" + TIMESTAMP_LITERAL + ", 'YYYY/MM/dd HH:mm ZZZZ')", VARCHAR, "2001/08/22 03:04 Asia/Kabul"); + assertFunction("format_datetime(" + TIMESTAMP_LITERAL + ", 'YYYY/MM/dd HH:mm ZZZZ')", VARCHAR, "2001/08/22 03:04 " + DATE_TIME_ZONE.getID()); } @Test @@ -52,7 +52,7 @@ public void testLocalTime() .setStartTime(new DateTime(2017, 3, 1, 14, 30, 0, 0, DATE_TIME_ZONE).getMillis()) .build(); try (FunctionAssertions localAssertion = new FunctionAssertions(localSession)) { - localAssertion.assertFunctionString("LOCALTIME", TimeType.TIME, "14:30:00.000"); + localAssertion.assertFunctionString("LOCALTIME", TimeType.TIME, "13:30:00.000"); } } @@ -88,8 +88,8 @@ public void testCurrentTimestamp() .setStartTime(new DateTime(2017, 3, 1, 14, 30, 0, 0, DATE_TIME_ZONE).getMillis()) .build(); try (FunctionAssertions localAssertion = new FunctionAssertions(localSession)) { - localAssertion.assertFunctionString("CURRENT_TIMESTAMP", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 Asia/Kabul"); - localAssertion.assertFunctionString("NOW()", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 Asia/Kabul"); + localAssertion.assertFunctionString("CURRENT_TIMESTAMP", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 " + DATE_TIME_ZONE.getID()); + localAssertion.assertFunctionString("NOW()", TIMESTAMP_WITH_TIME_ZONE, "2017-03-01 14:30:00.000 " + DATE_TIME_ZONE.getID()); } } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java b/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java index 20eed5662fec..1a39f8ce804b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java @@ -71,6 +71,7 @@ import static com.facebook.presto.sql.planner.ExpressionInterpreter.expressionInterpreter; import static com.facebook.presto.sql.planner.ExpressionInterpreter.expressionOptimizer; import static com.facebook.presto.type.IntervalDayTimeType.INTERVAL_DAY_TIME; +import static com.facebook.presto.util.DateTimeZoneIndex.getDateTimeZone; import static io.airlift.slice.Slices.utf8Slice; import static java.lang.String.format; import static java.util.Collections.emptyList; @@ -370,7 +371,7 @@ public void testBetween() @Test public void testExtract() { - DateTime dateTime = new DateTime(2001, 8, 22, 3, 4, 5, 321, DateTimeZone.UTC); + DateTime dateTime = new DateTime(2001, 8, 22, 3, 4, 5, 321, getDateTimeZone(TEST_SESSION.getTimeZoneKey())); double seconds = dateTime.getMillis() / 1000.0; assertOptimizedEquals("extract (YEAR from from_unixtime(" + seconds + "))", "2001"); @@ -390,10 +391,10 @@ public void testExtract() assertOptimizedEquals("extract (QUARTER from bound_timestamp)", "3"); assertOptimizedEquals("extract (MONTH from bound_timestamp)", "8"); assertOptimizedEquals("extract (WEEK from bound_timestamp)", "34"); - assertOptimizedEquals("extract (DOW from bound_timestamp)", "3"); - assertOptimizedEquals("extract (DOY from bound_timestamp)", "234"); - assertOptimizedEquals("extract (DAY from bound_timestamp)", "22"); - assertOptimizedEquals("extract (HOUR from bound_timestamp)", "3"); + assertOptimizedEquals("extract (DOW from bound_timestamp)", "2"); + assertOptimizedEquals("extract (DOY from bound_timestamp)", "233"); + assertOptimizedEquals("extract (DAY from bound_timestamp)", "21"); + assertOptimizedEquals("extract (HOUR from bound_timestamp)", "16"); assertOptimizedEquals("extract (MINUTE from bound_timestamp)", "4"); assertOptimizedEquals("extract (SECOND from bound_timestamp)", "5"); // todo reenable when cast as timestamp with time zone is implemented diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java index 546da91df103..730fe79390ff 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java @@ -83,6 +83,7 @@ import static com.facebook.presto.testing.DateTimeTestingUtils.sqlTimestampOf; import static com.facebook.presto.type.JsonType.JSON; import static com.facebook.presto.type.UnknownType.UNKNOWN; +import static com.facebook.presto.util.DateTimeZoneIndex.getDateTimeZone; import static com.facebook.presto.util.StructuralTestUtil.mapType; import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; @@ -93,6 +94,7 @@ import static java.lang.Runtime.getRuntime; import static java.lang.String.format; import static java.util.concurrent.Executors.newFixedThreadPool; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.stream.Collectors.joining; import static java.util.stream.IntStream.range; import static org.joda.time.DateTimeZone.UTC; @@ -1401,7 +1403,14 @@ public void testExtract() millis = left.getMillis(); expected = callExtractFunction(TEST_SESSION.toConnectorSession(), millis, field); } - assertExecute(generateExpression("extract(" + field.toString() + " from from_unixtime(%s / 1000.0E0, 0, 0))", millis), BIGINT, expected); + DateTimeZone zone = getDateTimeZone(TEST_SESSION.getTimeZoneKey()); + long zoneOffsetMinutes = millis != null ? MILLISECONDS.toMinutes(zone.getOffset(millis)) : 0; + String expressionPattern = format( + "extract(%s from from_unixtime(%%s / 1000.0E0, %s, %s))", + field, + zoneOffsetMinutes / 60, + zoneOffsetMinutes % 60); + assertExecute(generateExpression(expressionPattern, millis), BIGINT, expected); } } diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestTimeBase.java b/presto-main/src/test/java/com/facebook/presto/type/TestTimeBase.java index f1258f0fdbb1..fa25fc16fb1c 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestTimeBase.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestTimeBase.java @@ -19,6 +19,7 @@ import com.facebook.presto.spi.type.SqlTimeWithTimeZone; import com.facebook.presto.spi.type.SqlTimestampWithTimeZone; import com.facebook.presto.spi.type.TimeZoneKey; +import com.facebook.presto.testing.TestingSession; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.testng.annotations.Test; @@ -40,7 +41,7 @@ public abstract class TestTimeBase extends AbstractTestFunctions { - protected static final TimeZoneKey TIME_ZONE_KEY = getTimeZoneKey("Europe/Berlin"); + protected static final TimeZoneKey TIME_ZONE_KEY = TestingSession.DEFAULT_TIME_ZONE_KEY; protected static final DateTimeZone DATE_TIME_ZONE = getDateTimeZone(TIME_ZONE_KEY); public TestTimeBase(boolean legacyTimestamp) diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestTimestampBase.java b/presto-main/src/test/java/com/facebook/presto/type/TestTimestampBase.java index 78e4af875581..55d1a86e1092 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestTimestampBase.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestTimestampBase.java @@ -18,6 +18,7 @@ import com.facebook.presto.spi.type.SqlTimeWithTimeZone; import com.facebook.presto.spi.type.SqlTimestampWithTimeZone; import com.facebook.presto.spi.type.TimeZoneKey; +import com.facebook.presto.testing.TestingSession; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.testng.annotations.Test; @@ -44,7 +45,7 @@ public abstract class TestTimestampBase extends AbstractTestFunctions { - protected static final TimeZoneKey TIME_ZONE_KEY = getTimeZoneKey("Europe/Berlin"); + protected static final TimeZoneKey TIME_ZONE_KEY = TestingSession.DEFAULT_TIME_ZONE_KEY; protected static final DateTimeZone DATE_TIME_ZONE = getDateTimeZone(TIME_ZONE_KEY); protected static final TimeZoneKey WEIRD_TIME_ZONE_KEY = getTimeZoneKeyForOffset(7 * 60 + 9); protected static final DateTimeZone WEIRD_ZONE = getDateTimeZone(WEIRD_TIME_ZONE_KEY); @@ -191,7 +192,7 @@ public void testCastToTimeWithTimeZone() new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 3, 4, 5, 321, DATE_TIME_ZONE).getMillis(), TIME_ZONE_KEY)); functionAssertions.assertFunctionString("cast(TIMESTAMP '2001-1-22 03:04:05.321' as time with time zone)", TIME_WITH_TIME_ZONE, - "03:04:05.321 Europe/Berlin"); + "03:04:05.321 " + DATE_TIME_ZONE.getID()); } @Test @@ -202,7 +203,7 @@ public void testCastToTimestampWithTimeZone() new SqlTimestampWithTimeZone(new DateTime(2001, 1, 22, 3, 4, 5, 321, DATE_TIME_ZONE).getMillis(), DATE_TIME_ZONE.toTimeZone())); functionAssertions.assertFunctionString("cast(TIMESTAMP '2001-1-22 03:04:05.321' as timestamp with time zone)", TIMESTAMP_WITH_TIME_ZONE, - "2001-01-22 03:04:05.321 Europe/Berlin"); + "2001-01-22 03:04:05.321 " + DATE_TIME_ZONE.getID()); } @Test diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java index 2f97e6dc2a04..77f83b48a8c9 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java @@ -239,7 +239,7 @@ public void testShardingByTemporalTimestampColumn() assertUpdate(joiner.toString(), format("VALUES(%s)", rows)); - MaterializedResult results = computeActual("SELECT format_datetime(col2, 'yyyyMMdd'), \"$shard_uuid\" FROM test_shard_temporal_timestamp"); + MaterializedResult results = computeActual("SELECT format_datetime(col2 AT TIME ZONE 'UTC', 'yyyyMMdd'), \"$shard_uuid\" FROM test_shard_temporal_timestamp"); assertEquals(results.getRowCount(), rows); // Each shard will only contain data of one date. @@ -272,7 +272,7 @@ public void testShardingByTemporalTimestampColumnBucketed() assertUpdate(joiner.toString(), format("VALUES(%s)", rows)); MaterializedResult results = computeActual("" + - "SELECT format_datetime(col2, 'yyyyMMdd'), \"$shard_uuid\" " + + "SELECT format_datetime(col2 AT TIME ZONE 'UTC', 'yyyyMMdd'), \"$shard_uuid\" " + "FROM test_shard_temporal_timestamp_bucketed"); assertEquals(results.getRowCount(), rows); diff --git a/presto-teradata-functions/src/test/java/com/facebook/presto/teradata/functions/TestTeradataDateFunctions.java b/presto-teradata-functions/src/test/java/com/facebook/presto/teradata/functions/TestTeradataDateFunctions.java index 0ee6cedb932f..1b626233e782 100644 --- a/presto-teradata-functions/src/test/java/com/facebook/presto/teradata/functions/TestTeradataDateFunctions.java +++ b/presto-teradata-functions/src/test/java/com/facebook/presto/teradata/functions/TestTeradataDateFunctions.java @@ -17,7 +17,6 @@ import com.facebook.presto.operator.scalar.AbstractTestFunctions; import com.facebook.presto.spi.type.DateType; import com.facebook.presto.spi.type.SqlDate; -import com.facebook.presto.spi.type.TimeZoneKey; import com.facebook.presto.spi.type.TimestampType; import org.joda.time.DateTime; import org.testng.annotations.BeforeClass; @@ -26,7 +25,6 @@ import java.time.LocalDate; import static com.facebook.presto.metadata.FunctionExtractor.extractFunctions; -import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKey; import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.testing.DateTimeTestingUtils.sqlTimestampOf; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; @@ -35,11 +33,9 @@ public class TestTeradataDateFunctions extends AbstractTestFunctions { - private static final TimeZoneKey TIME_ZONE_KEY = getTimeZoneKey("Asia/Kathmandu"); private static final Session SESSION = testSessionBuilder() .setCatalog("catalog") .setSchema("schema") - .setTimeZoneKey(TIME_ZONE_KEY) .build(); protected TestTeradataDateFunctions() diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestAggregations.java b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestAggregations.java index cc12e56a9aad..31b79f35c5aa 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestAggregations.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestAggregations.java @@ -665,20 +665,20 @@ public void testApproximateCountDistinct() assertQuery("SELECT approx_distinct(orderdate, 0.023) FROM orders", "SELECT 2443"); // test timestamp - assertQuery("SELECT approx_distinct(CAST(orderdate AS TIMESTAMP)) FROM orders", "SELECT 2384"); - assertQuery("SELECT approx_distinct(CAST(orderdate AS TIMESTAMP), 0.023) FROM orders", "SELECT 2384"); + assertQuery("SELECT approx_distinct(CAST(orderdate AS TIMESTAMP)) FROM orders", "SELECT 2347"); + assertQuery("SELECT approx_distinct(CAST(orderdate AS TIMESTAMP), 0.023) FROM orders", "SELECT 2347"); // test timestamp with time zone - assertQuery("SELECT approx_distinct(CAST((orderdate) AS TIMESTAMP WITH TIME ZONE)) FROM orders", "SELECT 2384"); - assertQuery("SELECT approx_distinct(CAST((orderdate) AS TIMESTAMP WITH TIME ZONE), 0.023) FROM orders", "SELECT 2384"); + assertQuery("SELECT approx_distinct(CAST(orderdate AS TIMESTAMP WITH TIME ZONE)) FROM orders", "SELECT 2347"); + assertQuery("SELECT approx_distinct(CAST(orderdate AS TIMESTAMP WITH TIME ZONE), 0.023) FROM orders", "SELECT 2347"); // test time - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME)) FROM orders", "SELECT 993"); - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME), 0.023) FROM orders", "SELECT 993"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME)) FROM orders", "SELECT 996"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME), 0.023) FROM orders", "SELECT 996"); // test time with time zone - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE)) FROM orders", "SELECT 993"); - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE), 0.023) FROM orders", "SELECT 993"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE)) FROM orders", "SELECT 996"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE), 0.023) FROM orders", "SELECT 996"); // test short decimal assertQuery("SELECT approx_distinct(CAST(custkey AS DECIMAL(18, 0))) FROM orders", "SELECT 990"); diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java index 04b98ce37a01..616d2e8a8d88 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java @@ -4753,28 +4753,28 @@ public void testShowStatsWithWhereFunctionCallFails() public void testAtTimeZone() { // TODO the expected values here are non-sensical due to https://github.com/prestodb/presto/issues/7122 - assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE INTERVAL '07:09' hour to minute"), zonedDateTime("2012-10-31 08:09:00.000 +07:09")); - assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'Asia/Oral'"), zonedDateTime("2012-10-31 06:00:00.000 Asia/Oral")); + assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE INTERVAL '07:09' hour to minute"), zonedDateTime("2012-10-30 18:09:00.000 +07:09")); + assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'Asia/Oral'"), zonedDateTime("2012-10-30 16:00:00.000 Asia/Oral")); assertEquals(computeScalar("SELECT MIN(x) AT TIME ZONE 'America/Chicago' FROM (VALUES TIMESTAMP '1970-01-01 00:01:00+00:00') t(x)"), zonedDateTime("1969-12-31 18:01:00.000 America/Chicago")); - assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE '+07:09'"), zonedDateTime("2012-10-31 08:09:00.000 +07:09")); + assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE '+07:09'"), zonedDateTime("2012-10-30 18:09:00.000 +07:09")); assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00 UTC' AT TIME ZONE 'America/Los_Angeles'"), zonedDateTime("2012-10-30 18:00:00.000 America/Los_Angeles")); - assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'America/Los_Angeles'"), zonedDateTime("2012-10-30 18:00:00.000 America/Los_Angeles")); + assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'America/Los_Angeles'"), zonedDateTime("2012-10-30 04:00:00.000 America/Los_Angeles")); assertEquals(computeActual("SELECT x AT TIME ZONE 'America/Los_Angeles' FROM (values TIMESTAMP '1970-01-01 00:01:00+00:00', TIMESTAMP '1970-01-01 08:01:00+08:00', TIMESTAMP '1969-12-31 16:01:00-08:00') t(x)").getOnlyColumnAsSet(), ImmutableSet.of(zonedDateTime("1969-12-31 16:01:00.000 America/Los_Angeles"))); assertEquals(computeActual("SELECT x AT TIME ZONE 'America/Los_Angeles' FROM (values TIMESTAMP '1970-01-01 00:01:00', TIMESTAMP '1970-01-01 08:01:00', TIMESTAMP '1969-12-31 16:01:00') t(x)").getOnlyColumn().collect(toList()), - ImmutableList.of(zonedDateTime("1969-12-31 16:01:00.000 America/Los_Angeles"), zonedDateTime("1970-01-01 00:01:00.000 America/Los_Angeles"), zonedDateTime("1969-12-31 08:01:00.000 America/Los_Angeles"))); + ImmutableList.of(zonedDateTime("1970-01-01 03:01:00.000 America/Los_Angeles"), zonedDateTime("1970-01-01 11:01:00.000 America/Los_Angeles"), zonedDateTime("1969-12-31 19:01:00.000 America/Los_Angeles"))); assertEquals(computeScalar("SELECT min(x) AT TIME ZONE 'America/Los_Angeles' FROM (values TIMESTAMP '1970-01-01 00:01:00+00:00', TIMESTAMP '1970-01-01 08:01:00+08:00', TIMESTAMP '1969-12-31 16:01:00-08:00') t(x)"), zonedDateTime("1969-12-31 16:01:00.000 America/Los_Angeles")); // with chained AT TIME ZONE - assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'America/Los_Angeles' AT TIME ZONE 'UTC'"), zonedDateTime("2012-10-31 01:00:00.000 UTC")); - assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'Asia/Tokyo' AT TIME ZONE 'America/Los_Angeles'"), zonedDateTime("2012-10-30 18:00:00.000 America/Los_Angeles")); - assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'America/Los_Angeles' AT TIME ZONE 'Asia/Shanghai'"), zonedDateTime("2012-10-31 09:00:00.000 Asia/Shanghai")); + assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'America/Los_Angeles' AT TIME ZONE 'UTC'"), zonedDateTime("2012-10-30 11:00:00.000 UTC")); + assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'Asia/Tokyo' AT TIME ZONE 'America/Los_Angeles'"), zonedDateTime("2012-10-30 04:00:00.000 America/Los_Angeles")); + assertEquals(computeScalar("SELECT TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'America/Los_Angeles' AT TIME ZONE 'Asia/Shanghai'"), zonedDateTime("2012-10-30 19:00:00.000 Asia/Shanghai")); assertEquals(computeScalar("SELECT min(x) AT TIME ZONE 'America/Los_Angeles' AT TIME ZONE 'UTC' FROM (values TIMESTAMP '1970-01-01 00:01:00+00:00', TIMESTAMP '1970-01-01 08:01:00+08:00', TIMESTAMP '1969-12-31 16:01:00-08:00') t(x)"), zonedDateTime("1970-01-01 00:01:00.000 UTC")); // with AT TIME ZONE in VALUES - assertEquals(computeScalar("SELECT * FROM (VALUES TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'Asia/Oral')"), zonedDateTime("2012-10-31 06:00:00.000 Asia/Oral")); + assertEquals(computeScalar("SELECT * FROM (VALUES TIMESTAMP '2012-10-31 01:00' AT TIME ZONE 'Asia/Oral')"), zonedDateTime("2012-10-30 16:00:00.000 Asia/Oral")); } private ZonedDateTime zonedDateTime(String value)