Skip to content

Commit

Permalink
Fix at_timezone(TIME WITH TIME ZONE)
Browse files Browse the repository at this point in the history
AT TIME ZONE does not take into account the fact, that TIME WITH TIME ZONE
does not represent real millisecond UTC in millisUtc field.
In fact, this field contains millisUtc assuming offset of TIME ZONE that was
valid on 1970-01-01. Such representation allows to simply represent local time
with time zone id. That means that TIME WITH TIME ZONE that represents eg.
'10:00:00.000 Asia/Kathmandu' will always represent this exact value.
However mapping of such value to other TZ (including UTC) may differ over time.
Eg. Asia/Kathmandu switched time zone offset on 1986 from +5:30 to +5:45.
Result of query like:
`SELECT time_with_tz_column FROM table;`
Will always be the same, however:
`SELECT time_with_tz_column AT TIME ZONE 'UTC' FROM table;`
Will yail differnet value in 1980 and 2000 after changes from this commit.

This is done to use current offset of TZ as function that stucked in 1970-01-01
offsets seems useless.
This is not perfect solution and is not fully aligned with standard, but standard
behavior cannot be achieved with current TIME WITH TIME ZONE representation, as
we are not able to read TZ offset from TIME WITH TIME ZONE itselve (at least not
in all cases).
  • Loading branch information
fiedukow authored and findepi committed Jun 27, 2018
1 parent 319d495 commit fe35527
Show file tree
Hide file tree
Showing 3 changed files with 159 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,13 +36,15 @@
import org.joda.time.format.ISODateTimeFormat;

import java.util.Locale;
import java.util.concurrent.TimeUnit;

import static com.facebook.presto.operator.scalar.QuarterOfYearDateTimeField.QUARTER_OF_YEAR;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static com.facebook.presto.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static com.facebook.presto.spi.type.DateTimeEncoding.unpackMillisUtc;
import static com.facebook.presto.spi.type.DateTimeEncoding.unpackZoneKey;
import static com.facebook.presto.spi.type.DateTimeEncoding.updateMillisUtc;
import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKey;
import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKeyForOffset;
import static com.facebook.presto.type.DateTimeOperators.modulo24Hour;
import static com.facebook.presto.util.DateTimeZoneIndex.extractZoneOffsetMinutes;
Expand Down Expand Up @@ -259,18 +261,18 @@ public static long fromISO8601Date(ConnectorSession session, @SqlType("varchar(x
@ScalarFunction(value = "at_timezone", hidden = true)
@LiteralParameters("x")
@SqlType(StandardTypes.TIME_WITH_TIME_ZONE)
public static long timeAtTimeZone(@SqlType(StandardTypes.TIME_WITH_TIME_ZONE) long timeWithTimeZone, @SqlType("varchar(x)") Slice zoneId)
public static long timeAtTimeZone(ConnectorSession session, @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) long timeWithTimeZone, @SqlType("varchar(x)") Slice zoneId)
{
return packDateTimeWithZone(unpackMillisUtc(timeWithTimeZone), zoneId.toStringUtf8());
return timeAtTimeZone(session, timeWithTimeZone, getTimeZoneKey(zoneId.toStringUtf8()));
}

@ScalarFunction(value = "at_timezone", hidden = true)
@SqlType(StandardTypes.TIME_WITH_TIME_ZONE)
public static long timeAtTimeZone(@SqlType(StandardTypes.TIME_WITH_TIME_ZONE) long timeWithTimeZone, @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long zoneOffset)
public static long timeAtTimeZone(ConnectorSession session, @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) long timeWithTimeZone, @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long zoneOffset)
{
checkCondition((zoneOffset % 60_000) == 0, INVALID_FUNCTION_ARGUMENT, "Invalid time zone offset interval: interval contains seconds");
int zoneOffsetMinutes = (int) (zoneOffset / 60_000);
return packDateTimeWithZone(unpackMillisUtc(timeWithTimeZone), getTimeZoneKeyForOffset(zoneOffsetMinutes));
checkCondition((zoneOffset % 60_000L) == 0L, INVALID_FUNCTION_ARGUMENT, "Invalid time zone offset interval: interval contains seconds");
long zoneOffsetMinutes = zoneOffset / 60_000L;
return timeAtTimeZone(session, timeWithTimeZone, getTimeZoneKeyForOffset(zoneOffsetMinutes));
}

@ScalarFunction(value = "at_timezone", hidden = true)
Expand All @@ -285,8 +287,8 @@ public static long timestampAtTimeZone(@SqlType(StandardTypes.TIMESTAMP_WITH_TIM
@SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE)
public static long timestampAtTimeZone(@SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long timestampWithTimeZone, @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long zoneOffset)
{
checkCondition((zoneOffset % 60_000) == 0, INVALID_FUNCTION_ARGUMENT, "Invalid time zone offset interval: interval contains seconds");
int zoneOffsetMinutes = (int) (zoneOffset / 60_000);
checkCondition((zoneOffset % 60_000L) == 0L, INVALID_FUNCTION_ARGUMENT, "Invalid time zone offset interval: interval contains seconds");
long zoneOffsetMinutes = zoneOffset / 60_000L;
return packDateTimeWithZone(unpackMillisUtc(timestampWithTimeZone), getTimeZoneKeyForOffset(zoneOffsetMinutes));
}

Expand Down Expand Up @@ -1229,6 +1231,33 @@ public static long parseDuration(@SqlType("varchar(x)") Slice duration)
}
}

private static long timeAtTimeZone(ConnectorSession session, long timeWithTimeZone, TimeZoneKey timeZoneKey)
{
DateTimeZone sourceTimeZone = getDateTimeZone(unpackZoneKey(timeWithTimeZone));
DateTimeZone targetTimeZone = getDateTimeZone(timeZoneKey);
long millis = unpackMillisUtc(timeWithTimeZone);

// STEP 1. Calculate source UTC millis in session start
millis += valueToSessionTimeZoneOffsetDiff(session.getStartTime(), sourceTimeZone);

// STEP 2. Calculate target UTC millis in 1970
millis -= valueToSessionTimeZoneOffsetDiff(session.getStartTime(), targetTimeZone);

// STEP 3. Make sure that value + offset is in 0 - 23:59:59.999
long localMillis = millis + targetTimeZone.getOffset(0);
// Loops up to 2 times in total
while (localMillis > TimeUnit.DAYS.toMillis(1)) {
millis -= TimeUnit.DAYS.toMillis(1);
localMillis -= TimeUnit.DAYS.toMillis(1);
}
while (localMillis < 0) {
millis += TimeUnit.DAYS.toMillis(1);
localMillis += TimeUnit.DAYS.toMillis(1);
}

return packDateTimeWithZone(millis, timeZoneKey);
}

// HACK WARNING!
// This method does calculate difference between timezone offset on current date (session start)
// and 1970-01-01 (same timezone). This is used to be able to avoid using fixed offset TZ for
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package com.facebook.presto.operator.scalar;

import com.facebook.presto.Session;
import com.facebook.presto.spi.StandardErrorCode;
import com.facebook.presto.spi.type.BigintType;
import com.facebook.presto.spi.type.DateType;
import com.facebook.presto.spi.type.SqlDate;
Expand Down Expand Up @@ -904,6 +905,122 @@ public void testDateTimeOutputString()
}

@Test
public void testTimeWithTimeZoneAtTimeZone()
{
// this test does use hidden at_timezone function as it is equivalent of using SQL syntax AT TIME ZONE
// but our test framework doesn't support that syntax directly.

Session oldKathmanduTimeZoneOffsetSession =
testSessionBuilder()
.setTimeZoneKey(TIME_ZONE_KEY)
.setStartTime(new DateTime(1980, 1, 1, 10, 0, 0, DATE_TIME_ZONE).getMillis())
.build();

TimeZoneKey europeWarsawTimeZoneKey = getTimeZoneKey("Europe/Warsaw");
DateTimeZone europeWarsawTimeZone = getDateTimeZone(europeWarsawTimeZoneKey);
Session europeWarsawSessionWinter =
testSessionBuilder()
.setTimeZoneKey(europeWarsawTimeZoneKey)
.setStartTime(new DateTime(2017, 1, 1, 10, 0, 0, europeWarsawTimeZone).getMillis())
.build();
try (FunctionAssertions europeWarsawAssertionsWinter = new FunctionAssertions(europeWarsawSessionWinter);
FunctionAssertions oldKathmanduTimeZoneOffsetAssertions = new FunctionAssertions(oldKathmanduTimeZoneOffsetSession)) {
long millisTenOClockWarsawWinter = new DateTime(1970, 1, 1, 9, 0, 0, 0, UTC_TIME_ZONE).getMillis();

// Simple shift to UTC
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '10:00 Europe/Warsaw', 'UTC')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(millisTenOClockWarsawWinter, UTC_KEY));

// Simple shift to fixed TZ
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '10:00 Europe/Warsaw', '+00:45')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(millisTenOClockWarsawWinter, getTimeZoneKey("+00:45")));

// Simple shift to geographical TZ
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '10:00 Europe/Warsaw', 'America/New_York')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(millisTenOClockWarsawWinter, getTimeZoneKey("America/New_York")));

// No shift but different time zone
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '10:00 Europe/Warsaw', 'Europe/Berlin')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(millisTenOClockWarsawWinter, getTimeZoneKey("Europe/Berlin")));

// Noop on UTC
assertFunction("at_timezone(TIME '10:00 UTC', 'UTC')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 10, 0, 0, 0, UTC_TIME_ZONE).getMillis(), TimeZoneKey.UTC_KEY));

// Noop on other TZ
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '10:00 Europe/Warsaw', 'Europe/Warsaw')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(millisTenOClockWarsawWinter, europeWarsawTimeZoneKey));

// Noop on other TZ on different session TZ
assertFunction("at_timezone(TIME '10:00 Europe/Warsaw', 'Europe/Warsaw')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(millisTenOClockWarsawWinter, europeWarsawTimeZoneKey));

// Shift through days back
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '2:00 Europe/Warsaw', 'America/New_York')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 20, 0, 0, 0, getDateTimeZone(getTimeZoneKey("America/New_York"))).getMillis(), getTimeZoneKey("America/New_York")));

// Shift through days forward
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '22:00 America/New_York', 'Europe/Warsaw')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 4, 0, 0, 0, europeWarsawTimeZone).getMillis(), europeWarsawTimeZoneKey));

// Shift backward on min value
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '00:00 +14:00', '+13:00')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 23, 0, 0, 0, getDateTimeZone(getTimeZoneKey("+13:00"))).getMillis(), getTimeZoneKey("+13:00")));

// Shift backward on min value
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '00:00 +14:00', '-14:00')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 20, 0, 0, 0, getDateTimeZone(getTimeZoneKey("-14:00"))).getMillis(), getTimeZoneKey("-14:00")));

// Shift backward on max value
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '23:59:59.999 +14:00', '+13:00')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 22, 59, 59, 999, getDateTimeZone(getTimeZoneKey("+13:00"))).getMillis(), getTimeZoneKey("+13:00")));

// Shift forward on max value
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '23:59:59.999 +14:00', '-14:00')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 19, 59, 59, 999, getDateTimeZone(getTimeZoneKey("-14:00"))).getMillis(), getTimeZoneKey("-14:00")));

// Asia/Kathmandu used +5:30 TZ until 1986 and than switched to +5:45
// This test checks if we do use offset of time zone valid currently and not the historical one
assertFunction("at_timezone(TIME '10:00 Asia/Kathmandu', 'UTC')",
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/Kathmandu
assertFunction("at_timezone(TIME '10:00 Asia/Kathmandu', 'Asia/Kathmandu')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 10, 0, 0, 0, DATE_TIME_ZONE).getMillis(), TIME_ZONE_KEY));

// 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.
oldKathmanduTimeZoneOffsetAssertions.assertFunction("at_timezone(TIME '10:00 Asia/Kathmandu', 'UTC')",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 4, 30, 0, 0, UTC_TIME_ZONE).getMillis(), TimeZoneKey.UTC_KEY));

// Check simple interval shift
europeWarsawAssertionsWinter.assertFunction("at_timezone(TIME '10:00 +01:00', INTERVAL '2' HOUR)",
TIME_WITH_TIME_ZONE,
new SqlTimeWithTimeZone(new DateTime(1970, 1, 1, 11, 0, 0, 0, getDateTimeZone(getTimeZoneKey("+02:00"))).getMillis(), getTimeZoneKey("+02:00")));

// Check to high interval shift
europeWarsawAssertionsWinter.assertInvalidFunction("at_timezone(TIME '10:00 +01:00', INTERVAL '60' HOUR)",
StandardErrorCode.INVALID_FUNCTION_ARGUMENT,
"Invalid offset minutes 3600");
}
}

public void testParseDuration()
{
assertFunction("parse_duration('1234 ns')", INTERVAL_DAY_TIME, new SqlIntervalDayTime(0, 0, 0, 0, 0));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.spi.type;

import com.facebook.presto.spi.PrestoException;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonValue;

Expand All @@ -28,6 +29,7 @@
import java.util.Set;
import java.util.TreeMap;

import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static java.lang.Character.isDigit;
import static java.lang.Math.abs;
import static java.lang.Math.max;
Expand Down Expand Up @@ -137,7 +139,9 @@ public static TimeZoneKey getTimeZoneKeyForOffset(long offsetMinutes)
return UTC_KEY;
}

checkArgument(offsetMinutes >= OFFSET_TIME_ZONE_MIN && offsetMinutes <= OFFSET_TIME_ZONE_MAX, "Invalid offset minutes %s", offsetMinutes);
if (!(offsetMinutes >= OFFSET_TIME_ZONE_MIN && offsetMinutes <= OFFSET_TIME_ZONE_MAX)) {
throw new PrestoException(INVALID_FUNCTION_ARGUMENT, String.format("Invalid offset minutes %s", offsetMinutes));
}
TimeZoneKey timeZoneKey = OFFSET_TIME_ZONE_KEYS[((int) offsetMinutes) - OFFSET_TIME_ZONE_MIN];
if (timeZoneKey == null) {
throw new TimeZoneNotSupportedException(zoneIdForOffset(offsetMinutes));
Expand Down

0 comments on commit fe35527

Please sign in to comment.