Skip to content

Commit

Permalink
Change default session zone in tests to non-UTC
Browse files Browse the repository at this point in the history
  • Loading branch information
findepi committed Oct 30, 2018
1 parent 96a8e24 commit ade4339
Show file tree
Hide file tree
Showing 15 changed files with 88 additions and 70 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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"));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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)
Expand All @@ -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;

Expand All @@ -81,7 +78,7 @@ public TestCassandraIntegrationSmokeTest()
public void setUp()
{
session = EmbeddedCassandra.getSession();
createTestTables(session, KEYSPACE, DATE_LOCAL);
createTestTables(session, KEYSPACE, DATE_TIME_LOCAL);
}

@Override
Expand All @@ -106,7 +103,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" +
Expand Down Expand Up @@ -237,17 +234,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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,21 +27,30 @@
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;

import java.util.Optional;

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
{
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()
Expand All @@ -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");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -85,10 +86,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 ZoneOffset WEIRD_ZONE = ZoneOffset.ofHoursMinutes(7, 9);
Expand All @@ -107,7 +108,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_DATE_TIME_ZONE);
protected static final String WEIRD_TIMESTAMP_LITERAL = "TIMESTAMP '2001-08-22 03:04:05.321 +07:09'";
Expand Down Expand Up @@ -278,9 +279,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());
Expand Down Expand Up @@ -560,6 +561,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)));
Expand All @@ -570,6 +572,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.plusNanos(3_000_000)));
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)));
Expand Down Expand Up @@ -1022,10 +1025,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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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");
}
}

Expand Down Expand Up @@ -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());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,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;
Expand Down Expand Up @@ -374,7 +375,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");
Expand All @@ -394,10 +395,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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,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;
Expand All @@ -95,6 +96,7 @@
import static java.lang.String.format;
import static java.util.Collections.singletonList;
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;
Expand Down Expand Up @@ -1460,7 +1462,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);
}
}

Expand Down
Loading

0 comments on commit ade4339

Please sign in to comment.