diff --git a/presto-accumulo/src/main/java/io/prestosql/plugin/accumulo/AccumuloClient.java b/presto-accumulo/src/main/java/io/prestosql/plugin/accumulo/AccumuloClient.java index 44f514141afa9..d703a95a88e58 100644 --- a/presto-accumulo/src/main/java/io/prestosql/plugin/accumulo/AccumuloClient.java +++ b/presto-accumulo/src/main/java/io/prestosql/plugin/accumulo/AccumuloClient.java @@ -40,6 +40,7 @@ import io.prestosql.spi.connector.TableNotFoundException; import io.prestosql.spi.predicate.Domain; import io.prestosql.spi.predicate.Marker.Bound; +import io.prestosql.spi.type.TimestampType; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Connector; @@ -183,6 +184,10 @@ private static void validateColumns(ConnectorTableMetadata meta) } } + if (column.getType() instanceof TimestampType && ((TimestampType) column.getType()).getPrecision() != 3) { + throw new PrestoException(NOT_SUPPORTED, format("%s type not supported", column.getType())); + } + columnNameBuilder.add(column.getName().toLowerCase(Locale.ENGLISH)); } diff --git a/presto-base-jdbc/src/test/java/io/prestosql/plugin/jdbc/TestJdbcQueryBuilder.java b/presto-base-jdbc/src/test/java/io/prestosql/plugin/jdbc/TestJdbcQueryBuilder.java index 82b26a2578928..2634f748089a8 100644 --- a/presto-base-jdbc/src/test/java/io/prestosql/plugin/jdbc/TestJdbcQueryBuilder.java +++ b/presto-base-jdbc/src/test/java/io/prestosql/plugin/jdbc/TestJdbcQueryBuilder.java @@ -402,7 +402,7 @@ public void testEmptyBuildSql() private static long toPrestoTimestamp(int year, int month, int day, int hour, int minute, int second) { - SqlTimestamp sqlTimestamp = DateTimeTestingUtils.sqlTimestampOf(year, month, day, hour, minute, second, 0, UTC, UTC_KEY, SESSION); + SqlTimestamp sqlTimestamp = DateTimeTestingUtils.sqlTimestampOf(3, year, month, day, hour, minute, second, 0, UTC, UTC_KEY, SESSION); if (SESSION.isLegacyTimestamp()) { return sqlTimestamp.getMillisUtc(); } diff --git a/presto-docs/src/main/sphinx/language/timestamp.rst b/presto-docs/src/main/sphinx/language/timestamp.rst index fd4c959293ca9..20bfb366151b6 100644 --- a/presto-docs/src/main/sphinx/language/timestamp.rst +++ b/presto-docs/src/main/sphinx/language/timestamp.rst @@ -33,7 +33,7 @@ Now, Presto treats ``TIMESTAMP`` values as a set of the following fields represe * ``DAY OF MONTH`` * ``HOUR OF DAY`` * ``MINUTE OF HOUR`` - * ``SECOND OF MINUTE`` - as ``DECIMAL(5, 3)`` + * ``SECOND OF MINUTE`` For that reason, a ``TIMESTAMP`` value is not linked with the session time zone in any way until a time zone is needed explicitly, such as when casting to a ``TIMESTAMP WITH TIME ZONE`` or diff --git a/presto-elasticsearch/src/test/java/io/prestosql/elasticsearch/BaseElasticsearchSmokeTest.java b/presto-elasticsearch/src/test/java/io/prestosql/elasticsearch/BaseElasticsearchSmokeTest.java index ca202aa9d2362..deb8fd33b67e0 100644 --- a/presto-elasticsearch/src/test/java/io/prestosql/elasticsearch/BaseElasticsearchSmokeTest.java +++ b/presto-elasticsearch/src/test/java/io/prestosql/elasticsearch/BaseElasticsearchSmokeTest.java @@ -92,7 +92,7 @@ public void testDescribeTable() .row("clerk", "varchar", "", "") .row("comment", "varchar", "", "") .row("custkey", "bigint", "", "") - .row("orderdate", "timestamp", "", "") + .row("orderdate", "timestamp(3)", "", "") .row("orderkey", "bigint", "", "") .row("orderpriority", "varchar", "", "") .row("orderstatus", "varchar", "", "") @@ -111,7 +111,7 @@ public void testShowCreateTable() " clerk varchar,\n" + " comment varchar,\n" + " custkey bigint,\n" + - " orderdate timestamp,\n" + + " orderdate timestamp(3),\n" + " orderkey bigint,\n" + " orderpriority varchar,\n" + " orderstatus varchar,\n" + diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java index b8d5afc01c234..69e39b2a3e5a6 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/AbstractTestHive.java @@ -4199,7 +4199,7 @@ else if (rowNumber % 19 == 1) { assertNull(row.getField(index)); } else { - SqlTimestamp expected = sqlTimestampOf(2011, 5, 6, 7, 8, 9, 123, timeZone, UTC_KEY, SESSION); + SqlTimestamp expected = sqlTimestampOf(3, 2011, 5, 6, 7, 8, 9, 123, timeZone, UTC_KEY, SESSION); assertEquals(row.getField(index), expected); } } diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveIntegrationSmokeTest.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveIntegrationSmokeTest.java index 6f620024f4a44..0993b9f9b7b75 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveIntegrationSmokeTest.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/TestHiveIntegrationSmokeTest.java @@ -5216,12 +5216,12 @@ public void testCollectColumnStatisticsOnCreateTable() " VALUES " + " (null, null, null, null, null, null, 'p1'), " + " (null, null, null, null, null, null, 'p1'), " + - " (true, BIGINT '1', DOUBLE '2.2', TIMESTAMP '2012-08-08 01:00', CAST('abc1' AS VARCHAR), CAST('bcd1' AS VARBINARY), 'p1')," + - " (false, BIGINT '0', DOUBLE '1.2', TIMESTAMP '2012-08-08 00:00', CAST('abc2' AS VARCHAR), CAST('bcd2' AS VARBINARY), 'p1')," + + " (true, BIGINT '1', DOUBLE '2.2', TIMESTAMP '2012-08-08 01:00:00.000', CAST('abc1' AS VARCHAR), CAST('bcd1' AS VARBINARY), 'p1')," + + " (false, BIGINT '0', DOUBLE '1.2', TIMESTAMP '2012-08-08 00:00:00.000', CAST('abc2' AS VARCHAR), CAST('bcd2' AS VARBINARY), 'p1')," + " (null, null, null, null, null, null, 'p2'), " + " (null, null, null, null, null, null, 'p2'), " + - " (true, BIGINT '2', DOUBLE '3.3', TIMESTAMP '2012-09-09 01:00', CAST('cba1' AS VARCHAR), CAST('dcb1' AS VARBINARY), 'p2'), " + - " (false, BIGINT '1', DOUBLE '2.3', TIMESTAMP '2012-09-09 00:00', CAST('cba2' AS VARCHAR), CAST('dcb2' AS VARBINARY), 'p2') " + + " (true, BIGINT '2', DOUBLE '3.3', TIMESTAMP '2012-09-09 01:00:00.000', CAST('cba1' AS VARCHAR), CAST('dcb1' AS VARBINARY), 'p2'), " + + " (false, BIGINT '1', DOUBLE '2.3', TIMESTAMP '2012-09-09 00:00:00.000', CAST('cba2' AS VARCHAR), CAST('dcb2' AS VARBINARY), 'p2') " + ") AS x (c_boolean, c_bigint, c_double, c_timestamp, c_varchar, c_varbinary, p_varchar)", tableName), 8); assertQuery(format("SHOW STATS FOR (SELECT * FROM %s WHERE p_varchar = 'p1')", tableName), @@ -6240,23 +6240,23 @@ private void createTableForAnalyzeTest(String tableName, boolean partitioned) // p_varchar = 'p1', p_bigint = BIGINT '7' " (null, null, null, null, null, null, 'p1', BIGINT '7'), " + " (null, null, null, null, null, null, 'p1', BIGINT '7'), " + - " (true, BIGINT '1', DOUBLE '2.2', TIMESTAMP '2012-08-08 01:00', 'abc1', X'bcd1', 'p1', BIGINT '7'), " + - " (false, BIGINT '0', DOUBLE '1.2', TIMESTAMP '2012-08-08 00:00', 'abc2', X'bcd2', 'p1', BIGINT '7'), " + + " (true, BIGINT '1', DOUBLE '2.2', TIMESTAMP '2012-08-08 01:00:00.000', 'abc1', X'bcd1', 'p1', BIGINT '7'), " + + " (false, BIGINT '0', DOUBLE '1.2', TIMESTAMP '2012-08-08 00:00:00.000', 'abc2', X'bcd2', 'p1', BIGINT '7'), " + // p_varchar = 'p2', p_bigint = BIGINT '7' " (null, null, null, null, null, null, 'p2', BIGINT '7'), " + " (null, null, null, null, null, null, 'p2', BIGINT '7'), " + - " (true, BIGINT '2', DOUBLE '3.3', TIMESTAMP '2012-09-09 01:00', 'cba1', X'dcb1', 'p2', BIGINT '7'), " + - " (false, BIGINT '1', DOUBLE '2.3', TIMESTAMP '2012-09-09 00:00', 'cba2', X'dcb2', 'p2', BIGINT '7'), " + + " (true, BIGINT '2', DOUBLE '3.3', TIMESTAMP '2012-09-09 01:00:00.000', 'cba1', X'dcb1', 'p2', BIGINT '7'), " + + " (false, BIGINT '1', DOUBLE '2.3', TIMESTAMP '2012-09-09 00:00:00.000', 'cba2', X'dcb2', 'p2', BIGINT '7'), " + // p_varchar = 'p3', p_bigint = BIGINT '8' " (null, null, null, null, null, null, 'p3', BIGINT '8'), " + " (null, null, null, null, null, null, 'p3', BIGINT '8'), " + - " (true, BIGINT '3', DOUBLE '4.4', TIMESTAMP '2012-10-10 01:00', 'bca1', X'cdb1', 'p3', BIGINT '8'), " + - " (false, BIGINT '2', DOUBLE '3.4', TIMESTAMP '2012-10-10 00:00', 'bca2', X'cdb2', 'p3', BIGINT '8'), " + + " (true, BIGINT '3', DOUBLE '4.4', TIMESTAMP '2012-10-10 01:00:00.000', 'bca1', X'cdb1', 'p3', BIGINT '8'), " + + " (false, BIGINT '2', DOUBLE '3.4', TIMESTAMP '2012-10-10 00:00:00.000', 'bca2', X'cdb2', 'p3', BIGINT '8'), " + // p_varchar = NULL, p_bigint = NULL - " (false, BIGINT '7', DOUBLE '7.7', TIMESTAMP '1977-07-07 07:07', 'efa1', X'efa1', NULL, NULL), " + - " (false, BIGINT '6', DOUBLE '6.7', TIMESTAMP '1977-07-07 07:06', 'efa2', X'efa2', NULL, NULL), " + - " (false, BIGINT '5', DOUBLE '5.7', TIMESTAMP '1977-07-07 07:05', 'efa3', X'efa3', NULL, NULL), " + - " (false, BIGINT '4', DOUBLE '4.7', TIMESTAMP '1977-07-07 07:04', 'efa4', X'efa4', NULL, NULL) " + + " (false, BIGINT '7', DOUBLE '7.7', TIMESTAMP '1977-07-07 07:07:00.000', 'efa1', X'efa1', NULL, NULL), " + + " (false, BIGINT '6', DOUBLE '6.7', TIMESTAMP '1977-07-07 07:06:00.000', 'efa2', X'efa2', NULL, NULL), " + + " (false, BIGINT '5', DOUBLE '5.7', TIMESTAMP '1977-07-07 07:05:00.000', 'efa3', X'efa3', NULL, NULL), " + + " (false, BIGINT '4', DOUBLE '4.7', TIMESTAMP '1977-07-07 07:04:00.000', 'efa4', X'efa4', NULL, NULL) " + ") AS x (c_boolean, c_bigint, c_double, c_timestamp, c_varchar, c_varbinary, p_varchar, p_bigint)", 16); if (partitioned) { diff --git a/presto-hive/src/test/java/io/prestosql/plugin/hive/parquet/ParquetTester.java b/presto-hive/src/test/java/io/prestosql/plugin/hive/parquet/ParquetTester.java index 5ec8793b0dd93..d2a7d8dc46640 100644 --- a/presto-hive/src/test/java/io/prestosql/plugin/hive/parquet/ParquetTester.java +++ b/presto-hive/src/test/java/io/prestosql/plugin/hive/parquet/ParquetTester.java @@ -531,7 +531,7 @@ private static Object getActualCursorValue(RecordCursor cursor, Type type, int f return new SqlDate(((Long) fieldFromCursor).intValue()); } if (TIMESTAMP.equals(type)) { - return new SqlTimestamp((long) fieldFromCursor, UTC_KEY); + return SqlTimestamp.legacyFromMillis(3, (long) fieldFromCursor, UTC_KEY); } return fieldFromCursor; } diff --git a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/TypeConverter.java b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/TypeConverter.java index d4e02afbd43b8..6794801637ff2 100644 --- a/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/TypeConverter.java +++ b/presto-iceberg/src/main/java/io/prestosql/plugin/iceberg/TypeConverter.java @@ -103,7 +103,6 @@ public final class TypeConverter .put(RealType.class, Types.FloatType.get()) .put(IntegerType.class, Types.IntegerType.get()) .put(TimeType.class, Types.TimeType.get()) - .put(TimestampType.class, Types.TimestampType.withoutZone()) .put(TimestampWithTimeZoneType.class, Types.TimestampType.withZone()) .put(VarcharType.class, Types.StringType.get()) .build(); @@ -177,6 +176,9 @@ public static org.apache.iceberg.types.Type toIcebergType(Type type) if (type instanceof MapType) { return fromMap((MapType) type); } + if (type.equals(TIMESTAMP)) { + return Types.TimestampType.withoutZone(); + } throw new PrestoException(NOT_SUPPORTED, "Type not supported for Iceberg: " + type.getDisplayName()); } diff --git a/presto-jdbc/src/main/java/io/prestosql/jdbc/PrestoResultSet.java b/presto-jdbc/src/main/java/io/prestosql/jdbc/PrestoResultSet.java index 051a97b90b51c..cc874c41a4e7c 100644 --- a/presto-jdbc/src/main/java/io/prestosql/jdbc/PrestoResultSet.java +++ b/presto-jdbc/src/main/java/io/prestosql/jdbc/PrestoResultSet.java @@ -52,6 +52,8 @@ import java.sql.Time; import java.sql.Timestamp; import java.sql.Types; +import java.time.LocalDateTime; +import java.time.ZoneId; import java.util.Calendar; import java.util.HashMap; import java.util.Iterator; @@ -65,8 +67,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.stream.Stream; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Throwables.throwIfUnchecked; import static com.google.common.base.Verify.verify; import static com.google.common.collect.Iterables.getOnlyElement; @@ -81,6 +86,27 @@ public class PrestoResultSet implements ResultSet { + private static final Pattern DATETIME_PATTERN = Pattern.compile("" + + "(?\\d\\d\\d\\d)-(?\\d{1,2})-(?\\d{1,2})" + + "(?: (?\\d{1,2}):(?\\d{1,2})(?::(?\\d{1,2})(?:\\.(?\\d+))?)?)?" + + "\\s*(?.+)?"); + + private static final long[] POWERS_OF_TEN = { + 1L, + 10L, + 100L, + 1000L, + 10_000L, + 100_000L, + 1_000_000L, + 10_000_000L, + 100_000_000L, + 1_000_000_000L, + 10_000_000_000L, + 100_000_000_000L, + 1000_000_000_000L + }; + static final DateTimeFormatter DATE_FORMATTER = ISODateTimeFormat.date(); static final DateTimeFormatter TIME_FORMATTER = DateTimeFormat.forPattern("HH:mm:ss.SSS"); static final DateTimeFormatter TIME_WITH_TIME_ZONE_FORMATTER = new DateTimeFormatterBuilder() @@ -93,6 +119,7 @@ public class PrestoResultSet .withOffsetParsed(); static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); + static final DateTimeFormatter TIMESTAMP_WITH_TIME_ZONE_FORMATTER = new DateTimeFormatterBuilder() .append(DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS ZZZ").getPrinter(), new DateTimeParser[] { @@ -329,9 +356,9 @@ private Timestamp getTimestamp(int columnIndex, DateTimeZone localTimeZone) } ColumnInfo columnInfo = columnInfo(columnIndex); - if (columnInfo.getColumnTypeName().equalsIgnoreCase("timestamp")) { + if (columnInfo.getColumnTypeSignature().getRawType().equalsIgnoreCase("timestamp")) { try { - return new Timestamp(TIMESTAMP_FORMATTER.withZone(localTimeZone).parseMillis(String.valueOf(value))); + return parseTimestamp(String.valueOf(value), localTimeZone); } catch (IllegalArgumentException e) { throw new SQLException("Invalid timestamp from server: " + value, e); @@ -1924,4 +1951,71 @@ private static List getColumnInfo(List columns) } return list.build(); } + + private static Timestamp parseTimestamp(String value, DateTimeZone localTimeZone) + { + Matcher matcher = DATETIME_PATTERN.matcher(value); + if (!matcher.matches() || matcher.group("timezone") != null) { + throw new IllegalArgumentException("Invalid timestamp: " + value); + } + + int year = Integer.parseInt(matcher.group("year")); + int month = Integer.parseInt(matcher.group("month")); + int day = Integer.parseInt(matcher.group("day")); + int hour = Integer.parseInt(matcher.group("hour")); + int minute = Integer.parseInt(matcher.group("minute")); + int second = Integer.parseInt(matcher.group("second")); + String fraction = matcher.group("fraction"); + + long fractionValue = 0; + int precision = 0; + if (fraction != null) { + precision = fraction.length(); + fractionValue = Long.parseLong(fraction); + } + + long epochSecond = LocalDateTime.of(year, month, day, hour, minute, second, 0) + .atZone(ZoneId.of(localTimeZone.getID())) + .toEpochSecond(); + + Timestamp timestamp = new Timestamp(epochSecond * 1000); + timestamp.setNanos((int) rescale(fractionValue, precision, 9)); + return timestamp; + } + + public static long rescale(long value, int fromPrecision, int toPrecision) + { + if (value < 0) { + throw new IllegalArgumentException("value must be >= 0"); + } + + if (fromPrecision <= toPrecision) { + value *= scaleFactor(fromPrecision, toPrecision); + } + else { + value = roundDiv(value, scaleFactor(toPrecision, fromPrecision)); + } + + return value; + } + + private static long scaleFactor(int fromPrecision, int toPrecision) + { + if (fromPrecision > toPrecision) { + throw new IllegalArgumentException("fromPrecision must be <= toPrecision"); + } + + return POWERS_OF_TEN[toPrecision - fromPrecision]; + } + + private static long roundDiv(long value, long factor) + { + checkArgument(factor > 0, "factor must be positive"); + + if (value >= 0) { + return (value + (factor / 2)) / factor; + } + + return (value - (factor / 2)) / factor; + } } diff --git a/presto-jdbc/src/test/java/io/prestosql/jdbc/TestJdbcResultSet.java b/presto-jdbc/src/test/java/io/prestosql/jdbc/TestJdbcResultSet.java index ea813b858fc40..14f49fcab8505 100644 --- a/presto-jdbc/src/test/java/io/prestosql/jdbc/TestJdbcResultSet.java +++ b/presto-jdbc/src/test/java/io/prestosql/jdbc/TestJdbcResultSet.java @@ -178,6 +178,20 @@ public void testObjectTypes() assertEquals(rs.getTimestamp(column), Timestamp.valueOf(LocalDateTime.of(2018, 2, 13, 13, 14, 15, 123_000_000))); }); + checkRepresentation("TIMESTAMP '2018-02-13 13:14:15.111111111111'", Types.TIMESTAMP, (rs, column) -> { + assertEquals(rs.getObject(column), Timestamp.valueOf(LocalDateTime.of(2018, 2, 13, 13, 14, 15, 111_111_111))); + assertThrows(() -> rs.getDate(column)); + assertThrows(() -> rs.getTime(column)); + assertEquals(rs.getTimestamp(column), Timestamp.valueOf(LocalDateTime.of(2018, 2, 13, 13, 14, 15, 111_111_111))); + }); + + checkRepresentation("TIMESTAMP '2018-02-13 13:14:15.555555555555'", Types.TIMESTAMP, (rs, column) -> { + assertEquals(rs.getObject(column), Timestamp.valueOf(LocalDateTime.of(2018, 2, 13, 13, 14, 15, 555_555_556))); + assertThrows(() -> rs.getDate(column)); + assertThrows(() -> rs.getTime(column)); + assertEquals(rs.getTimestamp(column), Timestamp.valueOf(LocalDateTime.of(2018, 2, 13, 13, 14, 15, 555_555_556))); + }); + // TODO https://github.com/prestosql/presto/issues/37 // TODO line 1:8: '1970-01-01 00:14:15.123' is not a valid timestamp literal; the expected values will pro // checkRepresentation("TIMESTAMP '1970-01-01 00:14:15.123'", Types.TIMESTAMP, (rs, column) -> { diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/KafkaLoader.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/KafkaLoader.java index 1bfff3688acec..dcdc76a3d16c3 100644 --- a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/KafkaLoader.java +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/KafkaLoader.java @@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicReference; import static com.google.common.base.Preconditions.checkState; +import static io.prestosql.operator.scalar.timestamp.VarcharToTimestampCast.castToLegacyShortTimestamp; import static io.prestosql.spi.type.BigintType.BIGINT; import static io.prestosql.spi.type.BooleanType.BOOLEAN; import static io.prestosql.spi.type.DateTimeEncoding.unpackMillisUtc; @@ -46,7 +47,6 @@ import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; -import static io.prestosql.util.DateTimeUtils.convertToLegacyTimestamp; import static io.prestosql.util.DateTimeUtils.convertToTimestampWithTimeZone; import static io.prestosql.util.DateTimeUtils.parseLegacyTime; import static java.util.Objects.requireNonNull; @@ -146,7 +146,7 @@ private Object convertValue(Object value, Type type) return ISO8601_FORMATTER.print(parseLegacyTime(timeZoneKey, (String) value)); } if (TIMESTAMP.equals(type)) { - return ISO8601_FORMATTER.print(convertToLegacyTimestamp(timeZoneKey, (String) value)); + return ISO8601_FORMATTER.print(castToLegacyShortTimestamp(TIMESTAMP.getPrecision(), timeZoneKey, (String) value)); } if (TIME_WITH_TIME_ZONE.equals(type) || TIMESTAMP_WITH_TIME_ZONE.equals(type)) { return ISO8601_FORMATTER.print(unpackMillisUtc(convertToTimestampWithTimeZone(timeZoneKey, (String) value))); diff --git a/presto-main/src/main/java/io/prestosql/metadata/FunctionRegistry.java b/presto-main/src/main/java/io/prestosql/metadata/FunctionRegistry.java index 9c0c99c814f00..e475f0045d1be 100644 --- a/presto-main/src/main/java/io/prestosql/metadata/FunctionRegistry.java +++ b/presto-main/src/main/java/io/prestosql/metadata/FunctionRegistry.java @@ -143,6 +143,44 @@ import io.prestosql.operator.scalar.VarbinaryFunctions; import io.prestosql.operator.scalar.WilsonInterval; import io.prestosql.operator.scalar.WordStemFunction; +import io.prestosql.operator.scalar.timestamp.DateAdd; +import io.prestosql.operator.scalar.timestamp.DateDiff; +import io.prestosql.operator.scalar.timestamp.DateFormat; +import io.prestosql.operator.scalar.timestamp.DateToTimestampCast; +import io.prestosql.operator.scalar.timestamp.DateTrunc; +import io.prestosql.operator.scalar.timestamp.ExtractDay; +import io.prestosql.operator.scalar.timestamp.ExtractDayOfWeek; +import io.prestosql.operator.scalar.timestamp.ExtractDayOfYear; +import io.prestosql.operator.scalar.timestamp.ExtractHour; +import io.prestosql.operator.scalar.timestamp.ExtractMillisecond; +import io.prestosql.operator.scalar.timestamp.ExtractMinute; +import io.prestosql.operator.scalar.timestamp.ExtractMonth; +import io.prestosql.operator.scalar.timestamp.ExtractQuarter; +import io.prestosql.operator.scalar.timestamp.ExtractSecond; +import io.prestosql.operator.scalar.timestamp.ExtractWeekOfYear; +import io.prestosql.operator.scalar.timestamp.ExtractYear; +import io.prestosql.operator.scalar.timestamp.ExtractYearOfWeek; +import io.prestosql.operator.scalar.timestamp.FormatDateTime; +import io.prestosql.operator.scalar.timestamp.LastDayOfMonth; +import io.prestosql.operator.scalar.timestamp.LocalTimestamp; +import io.prestosql.operator.scalar.timestamp.SequenceIntervalDayToSecond; +import io.prestosql.operator.scalar.timestamp.SequenceIntervalYearToMonth; +import io.prestosql.operator.scalar.timestamp.TimeToTimestampCast; +import io.prestosql.operator.scalar.timestamp.TimeWithTimezoneToTimestampCast; +import io.prestosql.operator.scalar.timestamp.TimestampDistinctFromOperator; +import io.prestosql.operator.scalar.timestamp.TimestampOperators; +import io.prestosql.operator.scalar.timestamp.TimestampToDateCast; +import io.prestosql.operator.scalar.timestamp.TimestampToJsonCast; +import io.prestosql.operator.scalar.timestamp.TimestampToTimeCast; +import io.prestosql.operator.scalar.timestamp.TimestampToTimeWithTimezoneCast; +import io.prestosql.operator.scalar.timestamp.TimestampToTimestampCast; +import io.prestosql.operator.scalar.timestamp.TimestampToTimestampWithTimezoneCast; +import io.prestosql.operator.scalar.timestamp.TimestampToVarcharCast; +import io.prestosql.operator.scalar.timestamp.TimestampWithTimezoneToTimestampCast; +import io.prestosql.operator.scalar.timestamp.ToIso8601; +import io.prestosql.operator.scalar.timestamp.ToUnixTime; +import io.prestosql.operator.scalar.timestamp.VarcharToTimestampCast; +import io.prestosql.operator.scalar.timestamp.WithTimeZone; import io.prestosql.operator.window.CumulativeDistributionFunction; import io.prestosql.operator.window.DenseRankFunction; import io.prestosql.operator.window.FirstValueFunction; @@ -180,7 +218,6 @@ import io.prestosql.type.SmallintOperators; import io.prestosql.type.TimeOperators; import io.prestosql.type.TimeWithTimeZoneOperators; -import io.prestosql.type.TimestampOperators; import io.prestosql.type.TimestampWithTimeZoneOperators; import io.prestosql.type.TinyintOperators; import io.prestosql.type.UnknownOperators; @@ -457,8 +494,6 @@ public FunctionRegistry(Metadata metadata, FeaturesConfig featuresConfig) .scalar(DateOperators.DateDistinctFromOperator.class) .scalars(TimeOperators.class) .scalar(TimeOperators.TimeDistinctFromOperator.class) - .scalars(TimestampOperators.class) - .scalar(TimestampOperators.TimestampDistinctFromOperator.class) .scalars(IntervalDayTimeOperators.class) .scalar(IntervalDayTimeOperators.IntervalDayTimeDistinctFromOperator.class) .scalars(IntervalYearMonthOperators.class) @@ -594,6 +629,62 @@ public FunctionRegistry(Metadata metadata, FeaturesConfig featuresConfig) .scalars(SetDigestOperators.class) .scalars(WilsonInterval.class); + // timestamp operators and functions + builder + .scalar(TimestampOperators.Equal.class) + .scalar(TimestampOperators.NotEqual.class) + .scalar(TimestampOperators.LessThan.class) + .scalar(TimestampOperators.LessThanOrEqual.class) + .scalar(TimestampOperators.GreaterThan.class) + .scalar(TimestampOperators.GreaterThanOrEqual.class) + .scalar(TimestampDistinctFromOperator.class) + .scalar(TimestampOperators.HashCode.class) + .scalar(TimestampOperators.Indeterminate.class) + .scalar(TimestampOperators.XxHash64Operator.class) + .scalar(TimestampOperators.TimestampPlusIntervalDayToSecond.class) + .scalar(TimestampOperators.IntervalDayToSecondPlusTimestamp.class) + .scalar(TimestampOperators.TimestampPlusIntervalYearToMonth.class) + .scalar(TimestampOperators.IntervalYearToMonthPlusTimestamp.class) + .scalar(TimestampOperators.TimestampMinusIntervalDayToSecond.class) + .scalar(TimestampOperators.TimestampMinusIntervalYearToMonth.class) + .scalar(TimestampOperators.TimestampMinusTimestamp.class) + .scalar(TimestampToTimestampCast.class) + .scalar(TimestampToTimeCast.class) + .scalar(TimestampToTimeWithTimezoneCast.class) + .scalar(TimestampToTimestampWithTimezoneCast.class) + .scalar(TimestampToDateCast.class) + .scalar(TimestampToVarcharCast.class) + .scalar(TimestampToJsonCast.class) + .scalar(DateToTimestampCast.class) + .scalar(TimeToTimestampCast.class) + .scalar(TimeWithTimezoneToTimestampCast.class) + .scalar(TimestampWithTimezoneToTimestampCast.class) + .scalar(VarcharToTimestampCast.class) + .scalar(LocalTimestamp.class) + .scalar(DateTrunc.class) + .scalar(ToUnixTime.class) + .scalar(ToIso8601.class) + .scalar(WithTimeZone.class) + .scalar(FormatDateTime.class) + .scalar(DateFormat.class) + .scalar(SequenceIntervalYearToMonth.class) + .scalar(SequenceIntervalDayToSecond.class) + .scalar(DateAdd.class) + .scalar(DateDiff.class) + .scalar(ExtractYear.class) + .scalar(ExtractQuarter.class) + .scalar(ExtractMonth.class) + .scalar(ExtractDay.class) + .scalar(ExtractHour.class) + .scalar(ExtractMinute.class) + .scalar(ExtractSecond.class) + .scalar(ExtractMillisecond.class) + .scalar(ExtractDayOfYear.class) + .scalar(ExtractDayOfWeek.class) + .scalar(ExtractWeekOfYear.class) + .scalar(ExtractYearOfWeek.class) + .scalar(LastDayOfMonth.class); + switch (featuresConfig.getRegexLibrary()) { case JONI: builder.scalars(JoniRegexpFunctions.class); diff --git a/presto-main/src/main/java/io/prestosql/metadata/MetadataManager.java b/presto-main/src/main/java/io/prestosql/metadata/MetadataManager.java index 9a1e48b122294..53719bff55442 100644 --- a/presto-main/src/main/java/io/prestosql/metadata/MetadataManager.java +++ b/presto-main/src/main/java/io/prestosql/metadata/MetadataManager.java @@ -33,6 +33,7 @@ import io.prestosql.spi.block.ByteArrayBlockEncoding; import io.prestosql.spi.block.DictionaryBlockEncoding; import io.prestosql.spi.block.Int128ArrayBlockEncoding; +import io.prestosql.spi.block.Int96ArrayBlockEncoding; import io.prestosql.spi.block.IntArrayBlockEncoding; import io.prestosql.spi.block.LazyBlockEncoding; import io.prestosql.spi.block.LongArrayBlockEncoding; @@ -201,6 +202,7 @@ public MetadataManager( addBlockEncoding(new ShortArrayBlockEncoding()); addBlockEncoding(new IntArrayBlockEncoding()); addBlockEncoding(new LongArrayBlockEncoding()); + addBlockEncoding(new Int96ArrayBlockEncoding()); addBlockEncoding(new Int128ArrayBlockEncoding()); addBlockEncoding(new DictionaryBlockEncoding()); addBlockEncoding(new ArrayBlockEncoding()); diff --git a/presto-main/src/main/java/io/prestosql/metadata/TypeRegistry.java b/presto-main/src/main/java/io/prestosql/metadata/TypeRegistry.java index e7306a30297c7..6010ba80a5995 100644 --- a/presto-main/src/main/java/io/prestosql/metadata/TypeRegistry.java +++ b/presto-main/src/main/java/io/prestosql/metadata/TypeRegistry.java @@ -57,7 +57,7 @@ import static io.prestosql.spi.type.SmallintType.SMALLINT; import static io.prestosql.spi.type.TimeType.TIME; import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TimestampParametricType.TIMESTAMP; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.TinyintType.TINYINT; import static io.prestosql.spi.type.VarbinaryType.VARBINARY; @@ -108,7 +108,6 @@ public TypeRegistry(FeaturesConfig featuresConfig) addType(DATE); addType(TIME); addType(TIME_WITH_TIME_ZONE); - addType(TIMESTAMP); addType(TIMESTAMP_WITH_TIME_ZONE); addType(INTERVAL_YEAR_MONTH); addType(INTERVAL_DAY_TIME); @@ -132,6 +131,7 @@ public TypeRegistry(FeaturesConfig featuresConfig) addParametricType(MAP); addParametricType(FUNCTION); addParametricType(QDIGEST); + addParametricType(TIMESTAMP); parametricTypeCache = CacheBuilder.newBuilder() .maximumSize(1000) diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/DateTimeFunctions.java b/presto-main/src/main/java/io/prestosql/operator/scalar/DateTimeFunctions.java index 8114b895de6d4..193fd81be632d 100644 --- a/presto-main/src/main/java/io/prestosql/operator/scalar/DateTimeFunctions.java +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/DateTimeFunctions.java @@ -24,7 +24,6 @@ import io.prestosql.spi.function.SqlType; import io.prestosql.spi.type.StandardTypes; import io.prestosql.spi.type.TimeZoneKey; -import io.prestosql.type.TimestampOperators; import org.joda.time.DateTime; import org.joda.time.DateTimeField; import org.joda.time.DateTimeZone; @@ -149,20 +148,8 @@ public static long currentTimestamp(ConnectorSession session) return packDateTimeWithZone(session.getStart().toEpochMilli(), session.getTimeZoneKey()); } - @Description("Current timestamp without time zone") - @ScalarFunction("localtimestamp") - @SqlType(StandardTypes.TIMESTAMP) - public static long localTimestamp(ConnectorSession session) - { - if (session.isLegacyTimestamp()) { - return session.getStart().toEpochMilli(); - } - ISOChronology localChronology = getChronology(session.getTimeZoneKey()); - return localChronology.getZone().convertUTCToLocal(session.getStart().toEpochMilli()); - } - @ScalarFunction("from_unixtime") - @SqlType(StandardTypes.TIMESTAMP) + @SqlType("timestamp(3)") public static long fromUnixTime(@SqlType(StandardTypes.DOUBLE) double unixTime) { return Math.round(unixTime * 1000); @@ -190,13 +177,6 @@ public static long fromUnixTime(@SqlType(StandardTypes.DOUBLE) double unixTime, return packDateTimeWithZone(Math.round(unixTime * 1000), zoneId.toStringUtf8()); } - @ScalarFunction("to_unixtime") - @SqlType(StandardTypes.DOUBLE) - public static double toUnixTime(@SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - return timestamp / 1000.0; - } - @ScalarFunction("to_unixtime") @SqlType(StandardTypes.DOUBLE) public static double toUnixTimeFromTimestampWithTimeZone(@SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long timestampWithTimeZone) @@ -204,27 +184,6 @@ public static double toUnixTimeFromTimestampWithTimeZone(@SqlType(StandardTypes. return unpackMillisUtc(timestampWithTimeZone) / 1000.0; } - @ScalarFunction("to_iso8601") - @SqlType("varchar(29)") - // YYYY-MM-DDTHH:MM:SS.mmm is a standard notation, and it requires 23 characters. - // However extended notation with format ±(Y)+-MM-DDTHH:MM:SS.mmm is also acceptable and as - // the maximum year represented by 64bits timestamp is ~584944387 it may require up to 29 characters. - public static Slice toISO8601FromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - ISOChronology chronology; - - if (session.isLegacyTimestamp()) { - chronology = getChronology(session.getTimeZoneKey()); - } - else { - chronology = UTC_CHRONOLOGY; - } - - DateTimeFormatter formatter = ISODateTimeFormat.dateHourMinuteSecondMillis() - .withChronology(chronology); - return utf8Slice(formatter.print(timestamp)); - } - @ScalarFunction("to_iso8601") @SqlType("varchar(35)") // YYYY-MM-DDTHH:MM:SS.mmm+HH:MM is a standard notation, and it requires 29 characters. @@ -306,17 +265,6 @@ public static long timestampAtTimeZone(@SqlType(StandardTypes.TIMESTAMP_WITH_TIM return packDateTimeWithZone(unpackMillisUtc(timestampWithTimeZone), getTimeZoneKeyForOffset(zoneOffsetMinutes)); } - @ScalarFunction("with_timezone") - @LiteralParameters("x") - @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) - public static long withTimezone(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp, @SqlType("varchar(x)") Slice zoneId) - { - TimeZoneKey toTimeZoneKey = getTimeZoneKey(zoneId.toStringUtf8()); - DateTimeZone fromDateTimeZone = session.isLegacyTimestamp() ? getDateTimeZone(session.getTimeZoneKey()) : DateTimeZone.UTC; - DateTimeZone toDateTimeZone = getDateTimeZone(toTimeZoneKey); - return packDateTimeWithZone(fromDateTimeZone.getMillisKeepLocal(toDateTimeZone, timestamp), toTimeZoneKey); - } - @Description("Truncate to the specified precision in the session timezone") @ScalarFunction("date_trunc") @LiteralParameters("x") @@ -351,20 +299,6 @@ public static long truncateTimeWithTimeZone(@SqlType("varchar(x)") Slice unit, @ return updateMillisUtc(millis, timeWithTimeZone); } - @Description("Truncate to the specified precision in the session timezone") - @ScalarFunction("date_trunc") - @LiteralParameters("x") - @SqlType(StandardTypes.TIMESTAMP) - public static long truncateTimestamp(ConnectorSession session, @SqlType("varchar(x)") Slice unit, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getTimestampField(getChronology(session.getTimeZoneKey()), unit).roundFloor(timestamp); - } - else { - return getTimestampField(UTC_CHRONOLOGY, unit).roundFloor(timestamp); - } - } - @Description("Truncate to the specified precision") @ScalarFunction("date_trunc") @LiteralParameters("x") @@ -413,23 +347,6 @@ public static long addFieldValueTimeWithTimeZone( return updateMillisUtc(millis, timeWithTimeZone); } - @Description("Add the specified amount of time to the given timestamp") - @LiteralParameters("x") - @ScalarFunction("date_add") - @SqlType(StandardTypes.TIMESTAMP) - public static long addFieldValueTimestamp( - ConnectorSession session, - @SqlType("varchar(x)") Slice unit, - @SqlType(StandardTypes.BIGINT) long value, - @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getTimestampField(getChronology(session.getTimeZoneKey()), unit).add(timestamp, toIntExact(value)); - } - - return getTimestampField(UTC_CHRONOLOGY, unit).add(timestamp, toIntExact(value)); - } - @Description("Add the specified amount of time to the given timestamp") @LiteralParameters("x") @ScalarFunction("date_add") @@ -479,23 +396,6 @@ public static long diffTimeWithTimeZone( return getTimeField(unpackChronology(timeWithTimeZone1), unit).getDifferenceAsLong(unpackMillisUtc(timeWithTimeZone2), unpackMillisUtc(timeWithTimeZone1)); } - @Description("Difference of the given times in the given unit") - @ScalarFunction("date_diff") - @LiteralParameters("x") - @SqlType(StandardTypes.BIGINT) - public static long diffTimestamp( - ConnectorSession session, - @SqlType("varchar(x)") Slice unit, - @SqlType(StandardTypes.TIMESTAMP) long timestamp1, - @SqlType(StandardTypes.TIMESTAMP) long timestamp2) - { - if (session.isLegacyTimestamp()) { - return getTimestampField(getChronology(session.getTimeZoneKey()), unit).getDifferenceAsLong(timestamp2, timestamp1); - } - - return getTimestampField(UTC_CHRONOLOGY, unit).getDifferenceAsLong(timestamp2, timestamp1); - } - @Description("Difference of the given times in the given unit") @ScalarFunction("date_diff") @LiteralParameters("x") @@ -542,7 +442,7 @@ private static DateTimeField getTimeField(ISOChronology chronology, Slice unit) throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "'" + unitString + "' is not a valid Time field"); } - private static DateTimeField getTimestampField(ISOChronology chronology, Slice unit) + public static DateTimeField getTimestampField(ISOChronology chronology, Slice unit) { String unitString = unit.toStringUtf8().toLowerCase(ENGLISH); switch (unitString) { @@ -597,52 +497,6 @@ private static DateTime parseDateTimeHelper(DateTimeFormatter formatter, String } } - @Description("Formats the given time by the given format") - @ScalarFunction - @LiteralParameters("x") - @SqlType(StandardTypes.VARCHAR) - public static Slice formatDatetime(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp, @SqlType("varchar(x)") Slice formatString) - { - if (session.isLegacyTimestamp()) { - return formatDatetime(getChronology(session.getTimeZoneKey()), session.getLocale(), timestamp, formatString); - } - else { - if (datetimeFormatSpecifiesZone(formatString)) { - // Timezone is unknown for TIMESTAMP w/o TZ so it cannot be printed out. - throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "format_datetime for TIMESTAMP type, cannot use 'Z' nor 'z' in format, as this type does not contain TZ information"); - } - return formatDatetime(UTC_CHRONOLOGY, session.getLocale(), timestamp, formatString); - } - } - - /** - * Checks whether {@link DateTimeFormat} pattern contains time zone-related field. - */ - private static boolean datetimeFormatSpecifiesZone(Slice formatString) - { - boolean quoted = false; - for (char c : formatString.toStringUtf8().toCharArray()) { - if (quoted) { - if (c == '\'') { - quoted = false; - } - continue; - } - - switch (c) { - case 'z': - case 'Z': - return true; - case '\'': - // '' (two apostrophes) in a pattern denote single apostrophe and here we interpret this as "start quote" + "end quote". - // This has no impact on method's result value. - quoted = true; - break; - } - } - return false; - } - @Description("Formats the given time by the given format") @ScalarFunction("format_datetime") @LiteralParameters("x") @@ -668,19 +522,6 @@ private static Slice formatDatetime(ISOChronology chronology, Locale locale, lon } } - @ScalarFunction - @LiteralParameters("x") - @SqlType(StandardTypes.VARCHAR) - public static Slice dateFormat(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp, @SqlType("varchar(x)") Slice formatString) - { - if (session.isLegacyTimestamp()) { - return dateFormat(getChronology(session.getTimeZoneKey()), session.getLocale(), timestamp, formatString); - } - else { - return dateFormat(UTC_CHRONOLOGY, session.getLocale(), timestamp, formatString); - } - } - @ScalarFunction("date_format") @LiteralParameters("x") @SqlType(StandardTypes.VARCHAR) @@ -692,7 +533,7 @@ public static Slice dateFormatWithTimeZone( return dateFormat(unpackChronology(timestampWithTimeZone), session.getLocale(), unpackMillisUtc(timestampWithTimeZone), formatString); } - private static Slice dateFormat(ISOChronology chronology, Locale locale, long timestamp, Slice formatString) + public static Slice dateFormat(ISOChronology chronology, Locale locale, long timestamp, Slice formatString) { DateTimeFormatter formatter = DATETIME_FORMATTER_CACHE.get(formatString) .withChronology(chronology) @@ -703,7 +544,7 @@ private static Slice dateFormat(ISOChronology chronology, Locale locale, long ti @ScalarFunction @LiteralParameters({"x", "y"}) - @SqlType(StandardTypes.TIMESTAMP) + @SqlType("timestamp(3)") // TODO: increase precision? public static long dateParse(ConnectorSession session, @SqlType("varchar(x)") Slice dateTime, @SqlType("varchar(y)") Slice formatString) { DateTimeFormatter formatter = DATETIME_FORMATTER_CACHE.get(formatString) @@ -718,17 +559,6 @@ public static long dateParse(ConnectorSession session, @SqlType("varchar(x)") Sl } } - @Description("Millisecond of the second of the given timestamp") - @ScalarFunction("millisecond") - @SqlType(StandardTypes.BIGINT) - public static long millisecondFromTimestamp(@SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - // No need to check isLegacyTimestamp: - // * Under legacy semantics, the session zone matters. But a zone always has offset of whole minutes. - // * Under new semantics, timestamp is agnostic to the session zone. - return MILLISECOND_OF_SECOND.get(timestamp); - } - @Description("Millisecond of the second of the given timestamp") @ScalarFunction("millisecond") @SqlType(StandardTypes.BIGINT) @@ -766,17 +596,6 @@ public static long millisecondFromInterval(@SqlType(StandardTypes.INTERVAL_DAY_T return milliseconds % MILLISECONDS_IN_SECOND; } - @Description("Second of the minute of the given timestamp") - @ScalarFunction("second") - @SqlType(StandardTypes.BIGINT) - public static long secondFromTimestamp(@SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - // No need to check isLegacyTimestamp: - // * Under legacy semantics, the session zone matters. But a zone always has offset of whole minutes. - // * Under new semantics, timestamp is agnostic to the session zone. - return SECOND_OF_MINUTE.get(timestamp); - } - @Description("Second of the minute of the given timestamp") @ScalarFunction("second") @SqlType(StandardTypes.BIGINT) @@ -814,19 +633,6 @@ public static long secondFromInterval(@SqlType(StandardTypes.INTERVAL_DAY_TO_SEC return (milliseconds % MILLISECONDS_IN_MINUTE) / MILLISECONDS_IN_SECOND; } - @Description("Minute of the hour of the given timestamp") - @ScalarFunction("minute") - @SqlType(StandardTypes.BIGINT) - public static long minuteFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).minuteOfHour().get(timestamp); - } - else { - return MINUTE_OF_HOUR.get(timestamp); - } - } - @Description("Minute of the hour of the given timestamp") @ScalarFunction("minute") @SqlType(StandardTypes.BIGINT) @@ -864,19 +670,6 @@ public static long minuteFromInterval(@SqlType(StandardTypes.INTERVAL_DAY_TO_SEC return (milliseconds % MILLISECONDS_IN_HOUR) / MILLISECONDS_IN_MINUTE; } - @Description("Hour of the day of the given timestamp") - @ScalarFunction("hour") - @SqlType(StandardTypes.BIGINT) - public static long hourFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).hourOfDay().get(timestamp); - } - else { - return HOUR_OF_DAY.get(timestamp); - } - } - @Description("Hour of the day of the given timestamp") @ScalarFunction("hour") @SqlType(StandardTypes.BIGINT) @@ -914,19 +707,6 @@ public static long hourFromInterval(@SqlType(StandardTypes.INTERVAL_DAY_TO_SECON return (milliseconds % MILLISECONDS_IN_DAY) / MILLISECONDS_IN_HOUR; } - @Description("Day of the week of the given timestamp") - @ScalarFunction(value = "day_of_week", alias = "dow") - @SqlType(StandardTypes.BIGINT) - public static long dayOfWeekFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).dayOfWeek().get(timestamp); - } - else { - return DAY_OF_WEEK.get(timestamp); - } - } - @Description("Day of the week of the given timestamp") @ScalarFunction(value = "day_of_week", alias = "dow") @SqlType(StandardTypes.BIGINT) @@ -943,19 +723,6 @@ public static long dayOfWeekFromDate(@SqlType(StandardTypes.DATE) long date) return DAY_OF_WEEK.get(DAYS.toMillis(date)); } - @Description("Day of the month of the given timestamp") - @ScalarFunction(value = "day", alias = "day_of_month") - @SqlType(StandardTypes.BIGINT) - public static long dayFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).dayOfMonth().get(timestamp); - } - else { - return DAY_OF_MONTH.get(timestamp); - } - } - @Description("Day of the month of the given timestamp") @ScalarFunction(value = "day", alias = "day_of_month") @SqlType(StandardTypes.BIGINT) @@ -994,19 +761,6 @@ public static long lastDayOfMonthFromTimestampWithTimeZone(@SqlType(StandardType return MILLISECONDS.toDays(millis); } - @Description("Last day of the month of the given timestamp") - @ScalarFunction("last_day_of_month") - @SqlType(StandardTypes.DATE) - public static long lastDayOfMonthFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - long date = TimestampOperators.castToDate(session, timestamp); - return lastDayOfMonthFromDate(date); - } - long millis = UTC_CHRONOLOGY.monthOfYear().roundCeiling(timestamp + 1) - MILLISECONDS_IN_DAY; - return MILLISECONDS.toDays(millis); - } - @Description("Last day of the month of the given date") @ScalarFunction("last_day_of_month") @SqlType(StandardTypes.DATE) @@ -1016,19 +770,6 @@ public static long lastDayOfMonthFromDate(@SqlType(StandardTypes.DATE) long date return MILLISECONDS.toDays(millis); } - @Description("Day of the year of the given timestamp") - @ScalarFunction(value = "day_of_year", alias = "doy") - @SqlType(StandardTypes.BIGINT) - public static long dayOfYearFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).dayOfYear().get(timestamp); - } - else { - return DAY_OF_YEAR.get(timestamp); - } - } - @Description("Day of the year of the given timestamp") @ScalarFunction(value = "day_of_year", alias = "doy") @SqlType(StandardTypes.BIGINT) @@ -1045,19 +786,6 @@ public static long dayOfYearFromDate(@SqlType(StandardTypes.DATE) long date) return DAY_OF_YEAR.get(DAYS.toMillis(date)); } - @Description("Week of the year of the given timestamp") - @ScalarFunction(value = "week", alias = "week_of_year") - @SqlType(StandardTypes.BIGINT) - public static long weekFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).weekOfWeekyear().get(timestamp); - } - else { - return WEEK_OF_YEAR.get(timestamp); - } - } - @Description("Week of the year of the given timestamp") @ScalarFunction(value = "week", alias = "week_of_year") @SqlType(StandardTypes.BIGINT) @@ -1074,19 +802,6 @@ public static long weekFromDate(@SqlType(StandardTypes.DATE) long date) return WEEK_OF_YEAR.get(DAYS.toMillis(date)); } - @Description("Year of the ISO week of the given timestamp") - @ScalarFunction(value = "year_of_week", alias = "yow") - @SqlType(StandardTypes.BIGINT) - public static long yearOfWeekFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).weekyear().get(timestamp); - } - else { - return YEAR_OF_WEEK.get(timestamp); - } - } - @Description("Year of the ISO week of the given timestamp") @ScalarFunction(value = "year_of_week", alias = "yow") @SqlType(StandardTypes.BIGINT) @@ -1103,19 +818,6 @@ public static long yearOfWeekFromDate(@SqlType(StandardTypes.DATE) long date) return YEAR_OF_WEEK.get(DAYS.toMillis(date)); } - @Description("Month of the year of the given timestamp") - @ScalarFunction("month") - @SqlType(StandardTypes.BIGINT) - public static long monthFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).monthOfYear().get(timestamp); - } - else { - return MONTH_OF_YEAR.get(timestamp); - } - } - @Description("Month of the year of the given timestamp") @ScalarFunction("month") @SqlType(StandardTypes.BIGINT) @@ -1140,19 +842,6 @@ public static long monthFromInterval(@SqlType(StandardTypes.INTERVAL_YEAR_TO_MON return months % 12; } - @Description("Quarter of the year of the given timestamp") - @ScalarFunction("quarter") - @SqlType(StandardTypes.BIGINT) - public static long quarterFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return QUARTER_OF_YEAR.getField(getChronology(session.getTimeZoneKey())).get(timestamp); - } - else { - return QUARTER_OF_YEAR.getField(UTC_CHRONOLOGY).get(timestamp); - } - } - @Description("Quarter of the year of the given timestamp") @ScalarFunction("quarter") @SqlType(StandardTypes.BIGINT) @@ -1169,19 +858,6 @@ public static long quarterFromDate(@SqlType(StandardTypes.DATE) long date) return QUARTER.get(DAYS.toMillis(date)); } - @Description("Year of the given timestamp") - @ScalarFunction("year") - @SqlType(StandardTypes.BIGINT) - public static long yearFromTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).year().get(timestamp); - } - else { - return YEAR.get(timestamp); - } - } - @Description("Year of the given timestamp") @ScalarFunction("year") @SqlType(StandardTypes.BIGINT) diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/FormatFunction.java b/presto-main/src/main/java/io/prestosql/operator/scalar/FormatFunction.java index 592b3492d2c98..e5aa1f465381b 100644 --- a/presto-main/src/main/java/io/prestosql/operator/scalar/FormatFunction.java +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/FormatFunction.java @@ -29,6 +29,7 @@ import io.prestosql.spi.connector.ConnectorSession; import io.prestosql.spi.type.CharType; import io.prestosql.spi.type.DecimalType; +import io.prestosql.spi.type.TimestampType; import io.prestosql.spi.type.Type; import io.prestosql.spi.type.TypeSignature; import io.prestosql.sql.tree.QualifiedName; @@ -37,10 +38,8 @@ import java.math.BigDecimal; import java.time.Instant; import java.time.LocalDate; -import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; -import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.IllegalFormatException; import java.util.List; @@ -71,13 +70,13 @@ import static io.prestosql.spi.type.RealType.REAL; import static io.prestosql.spi.type.SmallintType.SMALLINT; import static io.prestosql.spi.type.TimeType.TIME; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.TinyintType.TINYINT; import static io.prestosql.spi.type.VarcharType.VARCHAR; import static io.prestosql.spi.type.Varchars.isVarcharType; import static io.prestosql.sql.analyzer.TypeSignatureProvider.fromTypes; import static io.prestosql.type.JsonType.JSON; +import static io.prestosql.type.Timestamps.toLocalDateTime; import static io.prestosql.type.UnknownType.UNKNOWN; import static io.prestosql.util.Failures.internalError; import static io.prestosql.util.Reflection.methodHandle; @@ -185,8 +184,8 @@ private static BiFunction valueConverter(Metada if (type.equals(TIMESTAMP_WITH_TIME_ZONE)) { return (session, block) -> toZonedDateTime(type.getLong(block, position)); } - if (type.equals(TIMESTAMP)) { - return (session, block) -> toLocalDateTime(session, type.getLong(block, position)); + if (type instanceof TimestampType) { + return (session, block) -> toLocalDateTime(((TimestampType) type), session, block, position); } if (type.equals(TIME)) { return (session, block) -> toLocalTime(session, type.getLong(block, position)); @@ -256,16 +255,6 @@ private static ZonedDateTime toZonedDateTime(long value) return ZonedDateTime.ofInstant(instant, zoneId); } - private static LocalDateTime toLocalDateTime(ConnectorSession session, long value) - { - Instant instant = Instant.ofEpochMilli(value); - if (session.isLegacyTimestamp()) { - ZoneId zoneId = ZoneId.of(session.getTimeZoneKey().getId()); - return LocalDateTime.ofInstant(instant, zoneId); - } - return LocalDateTime.ofInstant(instant, ZoneOffset.UTC); - } - private static LocalTime toLocalTime(ConnectorSession session, long value) { if (session.isLegacyTimestamp()) { diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/JsonOperators.java b/presto-main/src/main/java/io/prestosql/operator/scalar/JsonOperators.java index a081928cc7b3a..01f17cfffa70b 100644 --- a/presto-main/src/main/java/io/prestosql/operator/scalar/JsonOperators.java +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/JsonOperators.java @@ -52,11 +52,9 @@ import static io.prestosql.spi.type.StandardTypes.JSON; import static io.prestosql.spi.type.StandardTypes.REAL; import static io.prestosql.spi.type.StandardTypes.SMALLINT; -import static io.prestosql.spi.type.StandardTypes.TIMESTAMP; import static io.prestosql.spi.type.StandardTypes.TINYINT; import static io.prestosql.spi.type.StandardTypes.VARCHAR; import static io.prestosql.util.DateTimeUtils.printDate; -import static io.prestosql.util.DateTimeUtils.printTimestampWithoutTimeZone; import static io.prestosql.util.Failures.checkCondition; import static io.prestosql.util.JsonUtil.createJsonGenerator; import static io.prestosql.util.JsonUtil.createJsonParser; @@ -333,27 +331,6 @@ public static Slice castFromBoolean(@SqlType(BOOLEAN) boolean value) } } - @ScalarOperator(CAST) - @SqlType(JSON) - public static Slice castFromTimestamp(ConnectorSession session, @SqlType(TIMESTAMP) long value) - { - try { - SliceOutput output = new DynamicSliceOutput(25); - try (JsonGenerator jsonGenerator = createJsonGenerator(JSON_FACTORY, output)) { - if (session.isLegacyTimestamp()) { - jsonGenerator.writeString(printTimestampWithoutTimeZone(session.getTimeZoneKey(), value)); - } - else { - jsonGenerator.writeString(printTimestampWithoutTimeZone(value)); - } - } - return output.slice(); - } - catch (IOException e) { - throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast '%s' to %s", value, JSON)); - } - } - @ScalarOperator(CAST) @SqlType(JSON) public static Slice castFromDate(ConnectorSession session, @SqlType(DATE) long value) diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/SequenceFunction.java b/presto-main/src/main/java/io/prestosql/operator/scalar/SequenceFunction.java index 03bfdbb2b7cb3..a1c4329772769 100644 --- a/presto-main/src/main/java/io/prestosql/operator/scalar/SequenceFunction.java +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/SequenceFunction.java @@ -28,11 +28,9 @@ import java.util.concurrent.TimeUnit; import static io.prestosql.operator.scalar.DateTimeFunctions.diffDate; -import static io.prestosql.operator.scalar.DateTimeFunctions.diffTimestamp; import static io.prestosql.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.prestosql.spi.type.BigintType.BIGINT; import static io.prestosql.spi.type.DateType.DATE; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static io.prestosql.util.Failures.checkCondition; import static java.lang.Math.toIntExact; @@ -110,40 +108,6 @@ public static Block sequenceDateYearToMonth( return blockBuilder.build(); } - @ScalarFunction("sequence") - @SqlType("array(timestamp)") - public static Block sequenceTimestampDayToSecond( - @SqlType(StandardTypes.TIMESTAMP) long start, - @SqlType(StandardTypes.TIMESTAMP) long stop, - @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long step) - { - return fixedWidthSequence(start, stop, step, TIMESTAMP); - } - - @ScalarFunction("sequence") - @SqlType("array(timestamp)") - public static Block sequenceTimestampYearToMonth( - ConnectorSession session, - @SqlType(StandardTypes.TIMESTAMP) long start, - @SqlType(StandardTypes.TIMESTAMP) long stop, - @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long step) - { - checkValidStep(start, stop, step); - - int length = toIntExact(diffTimestamp(session, MONTH, start, stop) / step + 1); - checkMaxEntry(length); - - BlockBuilder blockBuilder = BIGINT.createBlockBuilder(null, length); - - int value = 0; - for (int i = 0; i < length; ++i) { - BIGINT.writeLong(blockBuilder, DateTimeOperators.timestampPlusIntervalYearToMonth(session, start, value)); - value += step; - } - - return blockBuilder.build(); - } - private static Block fixedWidthSequence(long start, long stop, long step, FixedWidthType type) { checkValidStep(start, stop, step); @@ -158,7 +122,7 @@ private static Block fixedWidthSequence(long start, long stop, long step, FixedW return blockBuilder.build(); } - private static void checkValidStep(long start, long stop, long step) + public static void checkValidStep(long start, long stop, long step) { checkCondition( step != 0, @@ -170,7 +134,7 @@ private static void checkValidStep(long start, long stop, long step) "sequence stop value should be greater than or equal to start value if step is greater than zero otherwise stop should be less than or equal to start"); } - private static void checkMaxEntry(int length) + public static void checkMaxEntry(int length) { checkCondition( length <= MAX_RESULT_ENTRIES, diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateAdd.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateAdd.java new file mode 100644 index 0000000000000..9a1427d460869 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateAdd.java @@ -0,0 +1,85 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.operator.scalar.DateTimeFunctions; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.getMicrosOfMilli; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; +import static java.lang.Math.toIntExact; + +@Description("Add the specified amount of time to the given timestamp") +@ScalarFunction("date_add") +public class DateAdd +{ + private DateAdd() {} + + @LiteralParameters({"x", "p"}) + @SqlType("timestamp(p)") + public static long add( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlType("varchar(x)") Slice unit, + @SqlType(StandardTypes.BIGINT) long value, + @SqlType("timestamp(p)") long timestamp) + { + long epochMillis = timestamp; + int microFraction = 0; + if (precision > 3) { + epochMillis = scaleEpochMicrosToMillis(timestamp); + microFraction = getMicrosOfMilli(timestamp); + } + + long result; + if (session.isLegacyTimestamp()) { + result = DateTimeFunctions.getTimestampField(getChronology(session.getTimeZoneKey()), unit).add(epochMillis, toIntExact(value)); + } + else { + result = DateTimeFunctions.getTimestampField(ISOChronology.getInstanceUTC(), unit).add(epochMillis, toIntExact(value)); + } + + if (precision <= 3) { + return round(result, (int) (3 - precision)); + } + + long epochMicros = scaleEpochMillisToMicros(result) + microFraction; + return round(epochMicros, (int) (6 - precision)); + } + + @LiteralParameters({"x", "p"}) + @SqlType("timestamp(p)") + public static LongTimestamp add( + ConnectorSession session, + @SqlType("varchar(x)") Slice unit, + @SqlType(StandardTypes.BIGINT) long value, + @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return new LongTimestamp( + add(6, session, unit, value, timestamp.getEpochMicros()), + timestamp.getPicosOfMicro()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateDiff.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateDiff.java new file mode 100644 index 0000000000000..5c0c7f2d97967 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateDiff.java @@ -0,0 +1,72 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.operator.scalar.DateTimeFunctions.getTimestampField; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Difference of the given times in the given unit") +@ScalarFunction("date_diff") +public class DateDiff +{ + private DateDiff() {} + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.BIGINT) + public static long diff( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlType("varchar(x)") Slice unit, + @SqlType("timestamp(p)") long timestamp1, + @SqlType("timestamp(p)") long timestamp2) + { + long epochMillis1 = timestamp1; + long epochMillis2 = timestamp2; + if (precision > 3) { + epochMillis1 = scaleEpochMicrosToMillis(timestamp1); + epochMillis2 = scaleEpochMicrosToMillis(timestamp2); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return getTimestampField(chronology, unit).getDifferenceAsLong(epochMillis2, epochMillis1); + } + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.BIGINT) + public static long diff( + ConnectorSession session, + @SqlType("varchar(x)") Slice unit, + @SqlType("timestamp(p)") LongTimestamp timestamp1, + @SqlType("timestamp(p)") LongTimestamp timestamp2) + { + // smallest unit of date_diff is "millisecond", so anything in the fraction is irrelevant + return diff(6, session, unit, timestamp1.getEpochMicros(), timestamp2.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateFormat.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateFormat.java new file mode 100644 index 0000000000000..8cc589f766c4d --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateFormat.java @@ -0,0 +1,61 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.operator.scalar.DateTimeFunctions.dateFormat; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarFunction +@Description("Formats the given timestamp by the given format") +public class DateFormat +{ + private DateFormat() {} + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.VARCHAR) + public static Slice format(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp, @SqlType("varchar(x)") Slice formatString) + { + // TODO: currently, date formatting only supports up to millis, so round to that unit + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(round(timestamp, 3)); + } + + if (session.isLegacyTimestamp()) { + return dateFormat(getChronology(session.getTimeZoneKey()), session.getLocale(), timestamp, formatString); + } + + return dateFormat(ISOChronology.getInstanceUTC(), session.getLocale(), timestamp, formatString); + } + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.VARCHAR) + public static Slice format(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp, @SqlType("varchar(x)") Slice formatString) + { + // Currently, date formatting only supports up to millis, so anything in the microsecond fraction is irrelevant + return format(6, session, timestamp.getEpochMicros(), formatString); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateToTimestampCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateToTimestampCast.java new file mode 100644 index 0000000000000..a45dc8c44c849 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateToTimestampCast.java @@ -0,0 +1,66 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import java.util.concurrent.TimeUnit; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarOperator(CAST) +public final class DateToTimestampCast +{ + private DateToTimestampCast() {} + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType(StandardTypes.DATE) long date) + { + long result; + if (session.isLegacyTimestamp()) { + long utcMillis = TimeUnit.DAYS.toMillis(date); + + // date is encoded as milliseconds at midnight in UTC + // convert to midnight in the session timezone + ISOChronology chronology = getChronology(session.getTimeZoneKey()); + result = utcMillis - chronology.getZone().getOffset(utcMillis); + } + else { + result = TimeUnit.DAYS.toMillis(date); + } + + if (precision > 3) { + return scaleEpochMillisToMicros(result); + } + + return result; + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp cast(ConnectorSession session, @SqlType(StandardTypes.DATE) long date) + { + return new LongTimestamp(cast(6, session, date), 0); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateTrunc.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateTrunc.java new file mode 100644 index 0000000000000..3f97b2b407263 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/DateTrunc.java @@ -0,0 +1,84 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.operator.scalar.DateTimeFunctions.getTimestampField; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Truncate to the specified precision in the session timezone") +@ScalarFunction("date_trunc") +public final class DateTrunc +{ + private DateTrunc() {} + + @LiteralParameters({"x", "p"}) + @SqlType("timestamp(p)") + public static long truncate( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlType("varchar(x)") Slice unit, + @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + long result; + if (session.isLegacyTimestamp()) { + result = getTimestampField(getChronology(session.getTimeZoneKey()), unit).roundFloor(timestamp); + } + else { + result = getTimestampField(ISOChronology.getInstanceUTC(), unit).roundFloor(timestamp); + } + + if (precision > 3) { + result = scaleEpochMillisToMicros(result); + } + + return result; + } + + @LiteralParameters({"x", "p"}) + @SqlType("timestamp(p)") + public static LongTimestamp truncate( + ConnectorSession session, + @SqlType("varchar(x)") Slice unit, + @SqlType("timestamp(p)") LongTimestamp timestamp) + { + long epochMillis = scaleEpochMicrosToMillis(timestamp.getEpochMicros()); + + long result; + if (session.isLegacyTimestamp()) { + result = getTimestampField(getChronology(session.getTimeZoneKey()), unit).roundFloor(epochMillis); + } + else { + result = getTimestampField(ISOChronology.getInstanceUTC(), unit).roundFloor(epochMillis); + } + + // smallest unit of truncation is "millisecond", so the fraction is always 0 + return new LongTimestamp(scaleEpochMillisToMicros(result), 0); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDay.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDay.java new file mode 100644 index 0000000000000..9e519b8505cfe --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDay.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Day of the month of the given timestamp") +@ScalarFunction(value = "day", alias = "day_of_month") +public class ExtractDay +{ + private ExtractDay() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.dayOfMonth().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDayOfWeek.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDayOfWeek.java new file mode 100644 index 0000000000000..24e5355bc6c84 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDayOfWeek.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Day of the week of the given timestamp") +@ScalarFunction(value = "day_of_week", alias = "dow") +public class ExtractDayOfWeek +{ + private ExtractDayOfWeek() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.dayOfWeek().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDayOfYear.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDayOfYear.java new file mode 100644 index 0000000000000..06530ff1dddc7 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractDayOfYear.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Day of the year of the given timestamp") +@ScalarFunction(value = "day_of_year", alias = "doy") +public class ExtractDayOfYear +{ + private ExtractDayOfYear() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.dayOfYear().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractHour.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractHour.java new file mode 100644 index 0000000000000..0996586f17025 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractHour.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Hour of the day of the given timestamp") +@ScalarFunction("hour") +public class ExtractHour +{ + private ExtractHour() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.hourOfDay().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMillisecond.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMillisecond.java new file mode 100644 index 0000000000000..b80ff2f7672fe --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMillisecond.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Millisecond of the second of the given timestamp") +@ScalarFunction("millisecond") +public class ExtractMillisecond +{ + private ExtractMillisecond() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.millisOfSecond().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMinute.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMinute.java new file mode 100644 index 0000000000000..75fe494a07701 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMinute.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Minute of the hour of the given timestamp") +@ScalarFunction("minute") +public class ExtractMinute +{ + private ExtractMinute() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.minuteOfHour().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMonth.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMonth.java new file mode 100644 index 0000000000000..6cf90689a87a8 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractMonth.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Month of the year of the given timestamp") +@ScalarFunction("month") +public class ExtractMonth +{ + private ExtractMonth() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.monthOfYear().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractQuarter.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractQuarter.java new file mode 100644 index 0000000000000..a5e1c4a46c1e7 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractQuarter.java @@ -0,0 +1,58 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.operator.scalar.QuarterOfYearDateTimeField.QUARTER_OF_YEAR; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Quarter of the year of the given timestamp") +@ScalarFunction("quarter") +public class ExtractQuarter +{ + private ExtractQuarter() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return QUARTER_OF_YEAR.getField(chronology).get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractSecond.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractSecond.java new file mode 100644 index 0000000000000..ed13aedbad956 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractSecond.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Second of the minute of the given timestamp") +@ScalarFunction("second") +public class ExtractSecond +{ + private ExtractSecond() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.secondOfMinute().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractWeekOfYear.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractWeekOfYear.java new file mode 100644 index 0000000000000..b841aa4f8fb1e --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractWeekOfYear.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Week of the year of the given timestamp") +@ScalarFunction(value = "week", alias = "week_of_year") +public class ExtractWeekOfYear +{ + private ExtractWeekOfYear() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.weekOfWeekyear().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractYear.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractYear.java new file mode 100644 index 0000000000000..1e170e786fcaa --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractYear.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Year of the given timestamp") +@ScalarFunction("year") +public class ExtractYear +{ + private ExtractYear() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.year().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractYearOfWeek.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractYearOfWeek.java new file mode 100644 index 0000000000000..d0848c7a3c2d4 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ExtractYearOfWeek.java @@ -0,0 +1,57 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Year of the ISO week of the given timestamp") +@ScalarFunction(value = "year_of_week", alias = "yow") +public class ExtractYearOfWeek +{ + private ExtractYearOfWeek() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology = ISOChronology.getInstanceUTC(); + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + + return chronology.weekyear().get(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long extract(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return extract(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/FormatDateTime.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/FormatDateTime.java new file mode 100644 index 0000000000000..862df8cdc17d2 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/FormatDateTime.java @@ -0,0 +1,108 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; +import org.joda.time.format.DateTimeFormat; + +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@Description("Formats the given time by the given format") +@ScalarFunction("format_datetime") +public class FormatDateTime +{ + private FormatDateTime() {} + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.VARCHAR) + public static Slice format(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp, @SqlType("varchar(x)") Slice formatString) + { + // TODO: currently, date formatting only supports up to millis, so we round to that unit + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(round(timestamp, 3)); + } + + ISOChronology chronology; + if (session.isLegacyTimestamp()) { + chronology = getChronology(session.getTimeZoneKey()); + } + else { + if (datetimeFormatSpecifiesZone(formatString)) { + // Timezone is unknown for TIMESTAMP w/o TZ so it cannot be printed out. + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "format_datetime for TIMESTAMP type, cannot use 'Z' nor 'z' in format, as this type does not contain TZ information"); + } + chronology = ISOChronology.getInstanceUTC(); + } + + try { + return utf8Slice(DateTimeFormat.forPattern(formatString.toStringUtf8()) + .withChronology(chronology) + .withLocale(session.getLocale()) + .print(timestamp)); + } + catch (Exception e) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, e); + } + } + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.VARCHAR) + public static Slice formatDatetime(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp, @SqlType("varchar(x)") Slice formatString) + { + // Currently, date formatting only supports up to millis, so anything in the microsecond fraction is irrelevant + return format(6, session, timestamp.getEpochMicros(), formatString); + } + + /** + * Checks whether {@link DateTimeFormat} pattern contains time zone-related field. + */ + private static boolean datetimeFormatSpecifiesZone(Slice formatString) + { + boolean quoted = false; + for (char c : formatString.toStringUtf8().toCharArray()) { + if (quoted) { + if (c == '\'') { + quoted = false; + } + continue; + } + + switch (c) { + case 'z': + case 'Z': + return true; + case '\'': + // '' (two apostrophes) in a pattern denote single apostrophe and here we interpret this as "start quote" + "end quote". + // This has no impact on method's result value. + quoted = true; + break; + } + } + return false; + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/LastDayOfMonth.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/LastDayOfMonth.java new file mode 100644 index 0000000000000..972d21e6976d6 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/LastDayOfMonth.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.Description; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.operator.scalar.DateTimeFunctions.lastDayOfMonthFromDate; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +@Description("Last day of the month of the given timestamp") +@ScalarFunction("last_day_of_month") +public class LastDayOfMonth +{ + private static final int MILLISECONDS_IN_DAY = 24 * 3600 * 1000; + + private LastDayOfMonth() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.DATE) + public static long lastDayOfMonth(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (session.isLegacyTimestamp()) { + long date = TimestampToDateCast.cast(precision, session, timestamp); + return lastDayOfMonthFromDate(date); + } + + long epochMillis = timestamp; + if (precision > 3) { + epochMillis = scaleEpochMicrosToMillis(timestamp); + } + + long millis = ISOChronology.getInstanceUTC().monthOfYear().roundCeiling(epochMillis + 1) - MILLISECONDS_IN_DAY; + return MILLISECONDS.toDays(millis); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.DATE) + public static long lastDayOfMonth(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return lastDayOfMonth(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/LocalTimestamp.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/LocalTimestamp.java new file mode 100644 index 0000000000000..d2626c27cadb9 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/LocalTimestamp.java @@ -0,0 +1,80 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlNullable; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.type.Timestamps; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; + +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; +import static io.prestosql.type.Timestamps.PICOSECONDS_PER_NANOSECOND; +import static io.prestosql.type.Timestamps.epochSecondToMicrosWithRounding; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; + +@ScalarFunction(value = "$localtimestamp", hidden = true) +public final class LocalTimestamp +{ + private LocalTimestamp() {} + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long localTimestamp( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlNullable @SqlType("timestamp(p)") Long dummy) // need a dummy value since the type inferencer can't bind type arguments exclusively from return type + { + Instant start = session.getStart(); + + if (!session.isLegacyTimestamp()) { + start = LocalDateTime.ofInstant(session.getStart(), session.getTimeZoneKey().getZoneId()) + .toInstant(ZoneOffset.UTC); + } + + long epochMicros = epochSecondToMicrosWithRounding(start.getEpochSecond(), ((long) start.getNano()) * PICOSECONDS_PER_NANOSECOND); + epochMicros = round(epochMicros, (int) (MAX_SHORT_PRECISION - precision)); + + if (precision <= 3) { + epochMicros = scaleEpochMicrosToMillis(epochMicros); + } + + return epochMicros; + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp localTimestamp( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlNullable @SqlType("timestamp(p)") LongTimestamp dummy) // need a dummy value since the type inferencer can't bind type arguments exclusively from return type + { + Instant start = session.getStart(); + + if (!session.isLegacyTimestamp()) { + start = LocalDateTime.ofInstant(session.getStart(), session.getTimeZoneKey().getZoneId()) + .toInstant(ZoneOffset.UTC); + } + + return Timestamps.longTimestamp(precision, start); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/SequenceIntervalDayToSecond.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/SequenceIntervalDayToSecond.java new file mode 100644 index 0000000000000..1ce0cb1bb080c --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/SequenceIntervalDayToSecond.java @@ -0,0 +1,90 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.BlockBuilder; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.LongTimestampType; +import io.prestosql.spi.type.ShortTimestampType; +import io.prestosql.spi.type.StandardTypes; + +import static io.prestosql.operator.scalar.SequenceFunction.checkMaxEntry; +import static io.prestosql.operator.scalar.SequenceFunction.checkValidStep; +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; +import static io.prestosql.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static java.lang.Math.multiplyExact; +import static java.lang.Math.toIntExact; + +@ScalarFunction("sequence") +public class SequenceIntervalDayToSecond +{ + // We need these because it's currently not possible to inject the fully-bound type into the methods that require them below + private static final ShortTimestampType SHORT_TYPE = new ShortTimestampType(0); + private static final LongTimestampType LONG_TYPE = new LongTimestampType(MAX_SHORT_PRECISION + 1); + + private SequenceIntervalDayToSecond() {} + + @LiteralParameters("p") + @SqlType("array(timestamp(p))") + public static Block sequence( + @LiteralParameter("p") long precision, + @SqlType("timestamp(p)") long start, + @SqlType("timestamp(p)") long stop, + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long step) + { + if (precision > 3) { + // scale to micros + step = multiplyExact(step, MICROSECONDS_PER_MILLISECOND); + } + + checkValidStep(start, stop, step); + + int length = toIntExact((stop - start) / step + 1L); + checkMaxEntry(length); + + BlockBuilder blockBuilder = SHORT_TYPE.createBlockBuilder(null, length); + for (long i = 0, value = start; i < length; ++i, value += step) { + SHORT_TYPE.writeLong(blockBuilder, value); + } + return blockBuilder.build(); + } + + @LiteralParameters("p") + @SqlType("array(timestamp(p))") + public static Block sequence( + @SqlType("timestamp(p)") LongTimestamp start, + @SqlType("timestamp(p)") LongTimestamp stop, + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long step) + { + step = multiplyExact(step, MICROSECONDS_PER_MILLISECOND); // scale to micros + + long startMicros = start.getEpochMicros(); + long stopMicros = stop.getEpochMicros(); + checkValidStep(startMicros, stopMicros, step); + + int length = toIntExact((stopMicros - startMicros) / step + 1L); + checkMaxEntry(length); + + BlockBuilder blockBuilder = LONG_TYPE.createBlockBuilder(null, length); + for (long i = 0, epochMicros = startMicros; i < length; ++i, epochMicros += step) { + LONG_TYPE.write(blockBuilder, epochMicros, start.getPicosOfMicro()); + } + return blockBuilder.build(); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/SequenceIntervalYearToMonth.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/SequenceIntervalYearToMonth.java new file mode 100644 index 0000000000000..dcfa6a57a6363 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/SequenceIntervalYearToMonth.java @@ -0,0 +1,97 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.prestosql.operator.scalar.timestamp.TimestampOperators.TimestampPlusIntervalYearToMonth; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.BlockBuilder; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.LongTimestampType; +import io.prestosql.spi.type.ShortTimestampType; +import io.prestosql.spi.type.StandardTypes; + +import static io.prestosql.operator.scalar.SequenceFunction.checkMaxEntry; +import static io.prestosql.operator.scalar.SequenceFunction.checkValidStep; +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; +import static java.lang.Math.toIntExact; + +@ScalarFunction("sequence") +public class SequenceIntervalYearToMonth +{ + // We need these because it's currently not possible to inject the fully-bound type into the methods that require them below + private static final ShortTimestampType SHORT_TYPE = new ShortTimestampType(0); + private static final LongTimestampType LONG_TYPE = new LongTimestampType(MAX_SHORT_PRECISION + 1); + + private static final Slice MONTH = Slices.utf8Slice("month"); + + private SequenceIntervalYearToMonth() {} + + @LiteralParameters("p") + @SqlType("array(timestamp(p))") + public static Block sequence( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlType("timestamp(p)") long start, + @SqlType("timestamp(p)") long stop, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long step) + { + checkValidStep(start, stop, step); + + int length = toIntExact(DateDiff.diff(precision, session, MONTH, start, stop) / step + 1); + checkMaxEntry(length); + + BlockBuilder blockBuilder = SHORT_TYPE.createBlockBuilder(null, length); + + int offset = 0; + for (int i = 0; i < length; ++i) { + long value = TimestampPlusIntervalYearToMonth.add(precision, session, start, offset); + SHORT_TYPE.writeLong(blockBuilder, value); + offset += step; + } + + return blockBuilder.build(); + } + + @LiteralParameters("p") + @SqlType("array(timestamp(p))") + public static Block sequence( + ConnectorSession session, + @SqlType("timestamp(p)") LongTimestamp start, + @SqlType("timestamp(p)") LongTimestamp stop, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long step) + { + checkValidStep(start.getEpochMicros(), stop.getEpochMicros(), step); + + int length = toIntExact(DateDiff.diff(session, MONTH, start, stop) / step + 1); + checkMaxEntry(length); + + BlockBuilder blockBuilder = LONG_TYPE.createBlockBuilder(null, length); + + int offset = 0; + for (int i = 0; i < length; ++i) { + LongTimestamp value = TimestampPlusIntervalYearToMonth.add(session, start, offset); + LONG_TYPE.writeObject(blockBuilder, value); + offset += step; + } + + return blockBuilder.build(); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimeToTimestampCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimeToTimestampCast.java new file mode 100644 index 0000000000000..de990af167af3 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimeToTimestampCast.java @@ -0,0 +1,54 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; + +@ScalarOperator(CAST) +public final class TimeToTimestampCast +{ + private TimeToTimestampCast() {} + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long cast(@LiteralParameter("p") long precision, @SqlType(StandardTypes.TIME) long time) + { + if (precision > 3) { + return scaleEpochMillisToMicros(time); + } + + if (precision < 3) { + // time is implicitly time(3), so truncate to expected precision + return round(time, (int) (3 - precision)); + } + + return time; + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp cast(@SqlType(StandardTypes.TIME) long time) + { + return new LongTimestamp(cast(6, time), 0); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimeWithTimezoneToTimestampCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimeWithTimezoneToTimestampCast.java new file mode 100644 index 0000000000000..e959393e1944a --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimeWithTimezoneToTimestampCast.java @@ -0,0 +1,77 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import java.time.Instant; +import java.time.ZoneOffset; +import java.time.temporal.ChronoField; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.spi.type.DateTimeEncoding.unpackMillisUtc; +import static io.prestosql.spi.type.DateTimeEncoding.unpackZoneKey; +import static io.prestosql.type.TimeWithTimeZoneOperators.REFERENCE_TIMESTAMP_UTC; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarOperator(CAST) +public final class TimeWithTimezoneToTimestampCast +{ + private TimeWithTimezoneToTimestampCast() {} + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) long time) + { + long epochMillis; + if (session.isLegacyTimestamp()) { + epochMillis = unpackMillisUtc(time); + } + else { + // This is hack that we need to use as the timezone interpretation depends on date (not only on time) + // TODO remove REFERENCE_TIMESTAMP_UTC when removing support for political time zones in TIME WITH TIME ZONE + long currentMillisOfDay = ChronoField.MILLI_OF_DAY.getFrom(Instant.ofEpochMilli(REFERENCE_TIMESTAMP_UTC).atZone(ZoneOffset.UTC)); + long timeMillisUtcInCurrentDay = REFERENCE_TIMESTAMP_UTC - currentMillisOfDay + unpackMillisUtc(time); + + ISOChronology chronology = getChronology(unpackZoneKey(time)); + epochMillis = unpackMillisUtc(time) + chronology.getZone().getOffset(timeMillisUtcInCurrentDay); + } + + if (precision > 3) { + return scaleEpochMillisToMicros(epochMillis); + } + + if (precision < 3) { + return round(epochMillis, (int) (3 - precision)); + } + + return epochMillis; + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp cast(ConnectorSession session, @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) long time) + { + return new LongTimestamp(cast(6, session, time), 0); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampDistinctFromOperator.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampDistinctFromOperator.java new file mode 100644 index 0000000000000..a0db937ecc739 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampDistinctFromOperator.java @@ -0,0 +1,108 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.block.Block; +import io.prestosql.spi.function.BlockIndex; +import io.prestosql.spi.function.BlockPosition; +import io.prestosql.spi.function.IsNull; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.LongTimestampType; +import io.prestosql.spi.type.ShortTimestampType; +import io.prestosql.spi.type.StandardTypes; + +import static io.prestosql.operator.scalar.timestamp.TimestampOperators.NotEqual.notEqual; +import static io.prestosql.spi.function.OperatorType.IS_DISTINCT_FROM; +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; + +@ScalarOperator(IS_DISTINCT_FROM) +public final class TimestampDistinctFromOperator +{ + // We need these because it's currently not possible to inject the fully-bound type into the methods that require them below + private static final ShortTimestampType SHORT_TYPE = new ShortTimestampType(0); + private static final LongTimestampType LONG_TYPE = new LongTimestampType(MAX_SHORT_PRECISION + 1); + + private TimestampDistinctFromOperator() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isDistinctFrom( + @SqlType("timestamp(p)") long left, + @IsNull boolean leftNull, + @SqlType("timestamp(p)") long right, + @IsNull boolean rightNull) + { + if (leftNull != rightNull) { + return true; + } + if (leftNull) { + return false; + } + return notEqual(left, right); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isDistinctFrom( + @BlockPosition @SqlType(value = "timestamp(p)", nativeContainerType = long.class) Block left, + @BlockIndex int leftPosition, + @BlockPosition @SqlType(value = "timestamp(p)", nativeContainerType = long.class) Block right, + @BlockIndex int rightPosition) + { + if (left.isNull(leftPosition) != right.isNull(rightPosition)) { + return true; + } + if (left.isNull(leftPosition)) { + return false; + } + return notEqual(SHORT_TYPE.getLong(left, leftPosition), SHORT_TYPE.getLong(right, rightPosition)); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isDistinctFromShort( + @SqlType("timestamp(p)") LongTimestamp left, + @IsNull boolean leftNull, + @SqlType("timestamp(p)") LongTimestamp right, + @IsNull boolean rightNull) + { + if (leftNull != rightNull) { + return true; + } + if (leftNull) { + return false; + } + return notEqual(left, right); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isDistinctFromLong( + @BlockPosition @SqlType(value = "timestamp(p)", nativeContainerType = LongTimestamp.class) Block left, + @BlockIndex int leftPosition, + @BlockPosition @SqlType(value = "timestamp(p)", nativeContainerType = LongTimestamp.class) Block right, + @BlockIndex int rightPosition) + { + if (left.isNull(leftPosition) != right.isNull(rightPosition)) { + return true; + } + if (left.isNull(leftPosition)) { + return false; + } + return !LONG_TYPE.equalTo(left, leftPosition, right, rightPosition); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampOperators.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampOperators.java new file mode 100644 index 0000000000000..41810b66c84a0 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampOperators.java @@ -0,0 +1,429 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.XxHash64; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.IsNull; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlNullable; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.LongTimestampType; +import io.prestosql.spi.type.ShortTimestampType; +import io.prestosql.spi.type.StandardTypes; +import io.prestosql.type.Constraint; +import org.joda.time.DateTimeField; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.spi.function.OperatorType.ADD; +import static io.prestosql.spi.function.OperatorType.EQUAL; +import static io.prestosql.spi.function.OperatorType.GREATER_THAN; +import static io.prestosql.spi.function.OperatorType.GREATER_THAN_OR_EQUAL; +import static io.prestosql.spi.function.OperatorType.HASH_CODE; +import static io.prestosql.spi.function.OperatorType.INDETERMINATE; +import static io.prestosql.spi.function.OperatorType.LESS_THAN; +import static io.prestosql.spi.function.OperatorType.LESS_THAN_OR_EQUAL; +import static io.prestosql.spi.function.OperatorType.NOT_EQUAL; +import static io.prestosql.spi.function.OperatorType.SUBTRACT; +import static io.prestosql.spi.function.OperatorType.XX_HASH_64; +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; +import static io.prestosql.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static io.prestosql.type.Timestamps.getMicrosOfMilli; +import static io.prestosql.type.Timestamps.rescale; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; +import static java.lang.Math.multiplyExact; + +public final class TimestampOperators +{ + private TimestampOperators() {} + + @ScalarOperator(EQUAL) + public static final class Equal + { + @LiteralParameters("p") + @SqlNullable + @SqlType(StandardTypes.BOOLEAN) + public static Boolean equal(@SqlType("timestamp(p)") long left, @SqlType("timestamp(p)") long right) + { + return left == right; + } + + @LiteralParameters("p") + @SqlNullable + @SqlType(StandardTypes.BOOLEAN) + public static Boolean equal(@SqlType("timestamp(p)") LongTimestamp left, @SqlType("timestamp(p)") LongTimestamp right) + { + return left.equals(right); + } + } + + @ScalarOperator(NOT_EQUAL) + public static final class NotEqual + { + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean notEqual(@SqlType("timestamp(p)") long left, @SqlType("timestamp(p)") long right) + { + return left != right; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean notEqual(@SqlType("timestamp(p)") LongTimestamp left, @SqlType("timestamp(p)") LongTimestamp right) + { + return !left.equals(right); + } + } + + @ScalarOperator(LESS_THAN) + public static final class LessThan + { + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean lessThan(@SqlType("timestamp(p)") long left, @SqlType("timestamp(p)") long right) + { + return left < right; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean lessThan(@SqlType("timestamp(p)") LongTimestamp left, @SqlType("timestamp(p)") LongTimestamp right) + { + return (left.getEpochMicros() < right.getEpochMicros()) || + ((left.getEpochMicros() == right.getEpochMicros()) && (left.getPicosOfMicro() < right.getPicosOfMicro())); + } + } + + @ScalarOperator(LESS_THAN_OR_EQUAL) + public static final class LessThanOrEqual + { + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean lessThanOrEqual(@SqlType("timestamp(p)") long left, @SqlType("timestamp(p)") long right) + { + return left <= right; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean lessThanOrEqual(@SqlType("timestamp(p)") LongTimestamp left, @SqlType("timestamp(p)") LongTimestamp right) + { + return left.getEpochMicros() < right.getEpochMicros() || + left.getEpochMicros() == right.getEpochMicros() && left.getPicosOfMicro() <= right.getPicosOfMicro(); + } + } + + @ScalarOperator(GREATER_THAN) + public static final class GreaterThan + { + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean greaterThan(@SqlType("timestamp(p)") long left, @SqlType("timestamp(p)") long right) + { + return left > right; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean greaterThan(@SqlType("timestamp(p)") LongTimestamp left, @SqlType("timestamp(p)") LongTimestamp right) + { + return !LessThanOrEqual.lessThanOrEqual(left, right); + } + } + + @ScalarOperator(GREATER_THAN_OR_EQUAL) + public static final class GreaterThanOrEqual + { + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean greaterThanOrEqual(@SqlType("timestamp(p)") long left, @SqlType("timestamp(p)") long right) + { + return left >= right; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean greaterThanOrEqual(@SqlType("timestamp(p)") LongTimestamp left, @SqlType("timestamp(p)") LongTimestamp right) + { + return !LessThan.lessThan(left, right); + } + } + + @ScalarOperator(HASH_CODE) + public static final class HashCode + { + @SqlType(StandardTypes.BIGINT) + @LiteralParameters("p") + public static long hashCode(@SqlType("timestamp(p)") long value) + { + return ShortTimestampType.hash(value); + } + + @SqlType(StandardTypes.BIGINT) + @LiteralParameters("p") + public static long hashCode(@SqlType("timestamp(p)") LongTimestamp value) + { + return LongTimestampType.hash(value); + } + } + + @ScalarOperator(INDETERMINATE) + public static final class Indeterminate + { + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean indeterminate(@SqlType("timestamp(p)") long value, @IsNull boolean isNull) + { + return isNull; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BOOLEAN) + public static boolean indeterminate(@SqlType("timestamp(p)") LongTimestamp value, @IsNull boolean isNull) + { + return isNull; + } + } + + @ScalarOperator(XX_HASH_64) + public static final class XxHash64Operator + { + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long xxHash64(@SqlType("timestamp(p)") long value) + { + return XxHash64.hash(value); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.BIGINT) + public static long xxHash64(@SqlType("timestamp(p)") LongTimestamp value) + { + return XxHash64.hash(value.getEpochMicros()) ^ XxHash64.hash(value.getPicosOfMicro()); + } + } + + @ScalarOperator(ADD) + public static final class TimestampPlusIntervalDayToSecond + { + @LiteralParameters({"p", "u"}) + @SqlType("timestamp(u)") + @Constraint(variable = "u", expression = "max(3, p)") // Interval is currently p = 3, so the minimum result precision is 3. + public static long add( + @LiteralParameter("p") long precision, + @SqlType("timestamp(p)") long timestamp, + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) + { + if (precision > 3) { + // scale to micros + interval = multiplyExact(interval, MICROSECONDS_PER_MILLISECOND); + } + + return timestamp + interval; + } + + @LiteralParameters({"p", "u"}) + @SqlType("timestamp(u)") + @Constraint(variable = "u", expression = "max(3, p)") // Interval is currently p = 3, so the minimum result precision is 3. + public static LongTimestamp add( + @SqlType("timestamp(p)") LongTimestamp timestamp, + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) + { + return new LongTimestamp(timestamp.getEpochMicros() + multiplyExact(interval, MICROSECONDS_PER_MILLISECOND), timestamp.getPicosOfMicro()); + } + } + + @ScalarOperator(ADD) + public static final class IntervalDayToSecondPlusTimestamp + { + @LiteralParameters({"p", "u"}) + @SqlType("timestamp(u)") + @Constraint(variable = "u", expression = "max(3, p)") // Interval is currently p = 3, so the minimum result precision is 3. + public static long add( + @LiteralParameter("p") long precision, + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval, + @SqlType("timestamp(p)") long timestamp) + { + return TimestampPlusIntervalDayToSecond.add(precision, timestamp, interval); + } + + @LiteralParameters({"p", "u"}) + @SqlType("timestamp(u)") + @Constraint(variable = "u", expression = "max(3, p)") // Interval is currently p = 3, so the minimum result precision is 3. + public static LongTimestamp add( + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval, + @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return TimestampPlusIntervalDayToSecond.add(timestamp, interval); + } + } + + @ScalarOperator(ADD) + public static final class TimestampPlusIntervalYearToMonth + { + private static final DateTimeField MONTH_OF_YEAR_UTC = ISOChronology.getInstanceUTC().monthOfYear(); + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long add( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlType("timestamp(p)") long timestamp, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) + { + long fractionMicros = 0; + if (precision > 3) { + fractionMicros = getMicrosOfMilli(timestamp); + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + long result; + if (session.isLegacyTimestamp()) { + result = getChronology(session.getTimeZoneKey()).monthOfYear().add(timestamp, interval); + } + else { + result = MONTH_OF_YEAR_UTC.add(timestamp, interval); + } + + if (precision > 3) { + return scaleEpochMillisToMicros(result) + fractionMicros; + } + + return result; + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp add( + ConnectorSession session, + @SqlType("timestamp(p)") LongTimestamp timestamp, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) + { + return new LongTimestamp( + add(6, session, timestamp.getEpochMicros(), interval), + timestamp.getPicosOfMicro()); + } + } + + @ScalarOperator(ADD) + public static final class IntervalYearToMonthPlusTimestamp + { + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long add( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval, + @SqlType("timestamp(p)") long timestamp) + { + return TimestampPlusIntervalYearToMonth.add(precision, session, timestamp, interval); + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp add( + ConnectorSession session, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval, + @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return TimestampPlusIntervalYearToMonth.add(session, timestamp, interval); + } + } + + @ScalarOperator(SUBTRACT) + public static final class TimestampMinusIntervalYearToMonth + { + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long subtract( + @LiteralParameter("p") long precision, + ConnectorSession session, + @SqlType("timestamp(p)") long timestamp, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) + { + return TimestampPlusIntervalYearToMonth.add(precision, session, timestamp, -interval); + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp subtract( + ConnectorSession session, + @SqlType("timestamp(p)") LongTimestamp timestamp, + @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) + { + return TimestampPlusIntervalYearToMonth.add(session, timestamp, -interval); + } + } + + @ScalarOperator(SUBTRACT) + public static final class TimestampMinusIntervalDayToSecond + { + @LiteralParameters({"p", "u"}) + @SqlType("timestamp(u)") + @Constraint(variable = "u", expression = "max(3, p)") // Interval is currently p = 3, so the minimum result precision is 3. + public static long subtract( + @LiteralParameter("p") long precision, + @SqlType("timestamp(p)") long timestamp, + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) + { + return TimestampPlusIntervalDayToSecond.add(precision, timestamp, -interval); + } + + @LiteralParameters({"p", "u"}) + @SqlType("timestamp(u)") + @Constraint(variable = "u", expression = "max(3, p)") // Interval is currently p = 3, so the minimum result precision is 3. + public static LongTimestamp subtract( + @SqlType("timestamp(p)") LongTimestamp timestamp, + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) + { + return TimestampPlusIntervalDayToSecond.add(timestamp, -interval); + } + } + + @ScalarOperator(SUBTRACT) + public static final class TimestampMinusTimestamp + { + @LiteralParameters("p") + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) + public static long subtract( + @LiteralParameter("p") long precision, + @SqlType("timestamp(p)") long left, + @SqlType("timestamp(p)") long right) + { + long interval = left - right; + + if (precision > 3) { + interval = round(interval, 3); + interval = rescale(interval, MAX_SHORT_PRECISION, 3); + } + + return interval; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) + public static long subtract( + @SqlType("timestamp(p)") LongTimestamp left, + @SqlType("timestamp(p)") LongTimestamp right) + { + return subtract(6, left.getEpochMicros(), right.getEpochMicros()); + } + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToDateCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToDateCast.java new file mode 100644 index 0000000000000..5ca0ee8de9614 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToDateCast.java @@ -0,0 +1,66 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import java.util.concurrent.TimeUnit; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarOperator(CAST) +@ScalarFunction("date") +public final class TimestampToDateCast +{ + private TimestampToDateCast() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.DATE) + public static long cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(timestamp); + } + + ISOChronology chronology; + if (session.isLegacyTimestamp()) { + // round down the current timestamp to days + chronology = getChronology(session.getTimeZoneKey()); + long date = chronology.dayOfYear().roundFloor(timestamp); + // date is currently midnight in timezone of the session + // convert to UTC + long millis = date + chronology.getZone().getOffset(date); + return TimeUnit.MILLISECONDS.toDays(millis); + } + + return TimeUnit.MILLISECONDS.toDays(timestamp); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.DATE) + public static long cast(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return cast(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToJsonCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToJsonCast.java new file mode 100644 index 0000000000000..0a848fa73a8a1 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToJsonCast.java @@ -0,0 +1,91 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import com.fasterxml.jackson.core.JsonGenerator; +import io.airlift.slice.DynamicSliceOutput; +import io.airlift.slice.Slice; +import io.airlift.slice.SliceOutput; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; + +import java.io.IOException; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; + +import static io.prestosql.spi.StandardErrorCode.INVALID_CAST_ARGUMENT; +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.spi.type.StandardTypes.JSON; +import static io.prestosql.type.Timestamps.formatTimestamp; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; +import static io.prestosql.util.JsonUtil.JSON_FACTORY; +import static io.prestosql.util.JsonUtil.createJsonGenerator; +import static java.lang.String.format; + +@ScalarOperator(CAST) +public final class TimestampToJsonCast +{ + private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss"); + + private TimestampToJsonCast() {} + + @LiteralParameters("p") + @SqlType(JSON) + public static Slice cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + long epochMicros = timestamp; + if (precision <= 3) { + epochMicros = scaleEpochMillisToMicros(timestamp); + } + + ZoneId zoneId = ZoneOffset.UTC; + if (session.isLegacyTimestamp()) { + zoneId = session.getTimeZoneKey().getZoneId(); + } + + return toJson(formatTimestamp((int) precision, epochMicros, 0, zoneId, TIMESTAMP_FORMATTER)); + } + + @LiteralParameters({"x", "p"}) + @SqlType(JSON) + public static Slice cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + ZoneId zoneId = ZoneOffset.UTC; + if (session.isLegacyTimestamp()) { + zoneId = session.getTimeZoneKey().getZoneId(); + } + + return toJson(formatTimestamp((int) precision, timestamp.getEpochMicros(), timestamp.getPicosOfMicro(), zoneId, TIMESTAMP_FORMATTER)); + } + + private static Slice toJson(String formatted) + { + try { + SliceOutput output = new DynamicSliceOutput(formatted.length() + 2); // 2 for the quotes + try (JsonGenerator jsonGenerator = createJsonGenerator(JSON_FACTORY, output)) { + jsonGenerator.writeString(formatted); + } + return output.slice(); + } + catch (IOException e) { + throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast '%s' to %s", formatted, JSON)); + } + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimeCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimeCast.java new file mode 100644 index 0000000000000..530c5460091c8 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimeCast.java @@ -0,0 +1,56 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.type.DateTimeOperators.modulo24Hour; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarOperator(CAST) +public final class TimestampToTimeCast +{ + private TimestampToTimeCast() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.TIME) + public static long cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long value) + { + if (precision > 3) { + value = scaleEpochMicrosToMillis(round(value, 3)); + } + + if (session.isLegacyTimestamp()) { + return modulo24Hour(getChronology(session.getTimeZoneKey()), value); + } + + return modulo24Hour(value); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.TIME) + public static long cast(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return cast(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimeWithTimezoneCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimeWithTimezoneCast.java new file mode 100644 index 0000000000000..e65996a4af0cf --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimeWithTimezoneCast.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone; +import static io.prestosql.type.DateTimeOperators.modulo24Hour; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarOperator(CAST) +public final class TimestampToTimeWithTimezoneCast +{ + private TimestampToTimeWithTimezoneCast() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) + public static long cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long value) + { + if (precision > 3) { + value = scaleEpochMicrosToMillis(round(value, 3)); + } + + if (session.isLegacyTimestamp()) { + int timeMillis = modulo24Hour(getChronology(session.getTimeZoneKey()), value); + return packDateTimeWithZone(timeMillis, session.getTimeZoneKey()); + } + + ISOChronology localChronology = getChronology(session.getTimeZoneKey()); + // This cast does treat TIMESTAMP as wall time in session TZ. This means that in order to get + // its UTC representation we need to shift the value by the offset of TZ. + return packDateTimeWithZone(localChronology.getZone().convertLocalToUTC(modulo24Hour(value), false), session.getTimeZoneKey()); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) + public static long cast(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return cast(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimestampCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimestampCast.java new file mode 100644 index 0000000000000..09716833c3a62 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimestampCast.java @@ -0,0 +1,108 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.spi.type.TimestampType.MAX_PRECISION; +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; +import static io.prestosql.type.Timestamps.PICOSECONDS_PER_MICROSECOND; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.roundToNearest; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; + +@ScalarOperator(CAST) +public final class TimestampToTimestampCast +{ + private TimestampToTimestampCast() {} + + @LiteralParameters({"sourcePrecision", "targetPrecision"}) + @SqlType("timestamp(targetPrecision)") + public static long shortToShort( + @LiteralParameter("sourcePrecision") long sourcePrecision, + @LiteralParameter("targetPrecision") long targetPrecision, + @SqlType("timestamp(sourcePrecision)") long value) + { + long epochMicros = value; + if (sourcePrecision <= 3) { + epochMicros = scaleEpochMillisToMicros(epochMicros); + } + + if (sourcePrecision <= targetPrecision) { + if (targetPrecision <= 3) { + return scaleEpochMicrosToMillis(epochMicros); + } + + return epochMicros; + } + + epochMicros = round(epochMicros, (int) (MAX_SHORT_PRECISION - targetPrecision)); + + if (targetPrecision <= 3) { + return scaleEpochMicrosToMillis(epochMicros); + } + + return epochMicros; + } + + @LiteralParameters({"sourcePrecision", "targetPrecision"}) + @SqlType("timestamp(targetPrecision)") + public static LongTimestamp shortToLong( + @LiteralParameter("sourcePrecision") long sourcePrecision, + @SqlType("timestamp(sourcePrecision)") long value) + { + long micros = value; + if (sourcePrecision <= 3) { + micros = scaleEpochMillisToMicros(value); + } + + return new LongTimestamp(micros, 0); + } + + @LiteralParameters({"sourcePrecision", "targetPrecision"}) + @SqlType("timestamp(targetPrecision)") + public static long longToShort( + @LiteralParameter("targetPrecision") long targetPrecision, + @SqlType("timestamp(sourcePrecision)") LongTimestamp value) + { + long epochMicros = value.getEpochMicros(); + if (targetPrecision <= 3) { + return scaleEpochMicrosToMillis(round(epochMicros, (int) (6 - targetPrecision))); + } + else if (targetPrecision < MAX_SHORT_PRECISION) { + return round(epochMicros, (int) (MAX_SHORT_PRECISION - targetPrecision)); + } + + if (roundToNearest(value.getPicosOfMicro(), PICOSECONDS_PER_MICROSECOND) == PICOSECONDS_PER_MICROSECOND) { + epochMicros++; + } + + return epochMicros; + } + + @LiteralParameters({"sourcePrecision", "targetPrecision"}) + @SqlType("timestamp(targetPrecision)") + public static LongTimestamp longToLong( + @LiteralParameter("targetPrecision") long targetPrecision, + @SqlType("timestamp(sourcePrecision)") LongTimestamp value) + { + return new LongTimestamp(value.getEpochMicros(), (int) round(value.getPicosOfMicro(), (int) (MAX_PRECISION - targetPrecision))); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimestampWithTimezoneCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimestampWithTimezoneCast.java new file mode 100644 index 0000000000000..e857e60512942 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToTimestampWithTimezoneCast.java @@ -0,0 +1,60 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarOperator(CAST) +public final class TimestampToTimestampWithTimezoneCast +{ + private TimestampToTimestampWithTimezoneCast() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) + public static long cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long value) + { + if (precision > 3) { + value = scaleEpochMicrosToMillis(round(value, 3)); + } + + if (session.isLegacyTimestamp()) { + return packDateTimeWithZone(value, session.getTimeZoneKey()); + } + + ISOChronology localChronology = getChronology(session.getTimeZoneKey()); + // This cast does treat TIMESTAMP as wall time in session TZ. This means that in order to get + // its UTC representation we need to shift the value by the offset of TZ. + return packDateTimeWithZone(localChronology.getZone().convertLocalToUTC(value, false), session.getTimeZoneKey()); + } + + @LiteralParameters("p") + @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) + public static long cast(ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + return cast(6, session, timestamp.getEpochMicros()); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToVarcharCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToVarcharCast.java new file mode 100644 index 0000000000000..4e8dfe9ee0dfb --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampToVarcharCast.java @@ -0,0 +1,68 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.type.Timestamps; + +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; + +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; + +@ScalarOperator(CAST) +public final class TimestampToVarcharCast +{ + private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss"); + + private TimestampToVarcharCast() {} + + @LiteralParameters({"x", "p"}) + @SqlType("varchar(x)") + public static Slice cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + long epochMicros = timestamp; + if (precision <= 3) { + epochMicros = scaleEpochMillisToMicros(timestamp); + } + + ZoneId zoneId = ZoneOffset.UTC; + if (session.isLegacyTimestamp()) { + zoneId = session.getTimeZoneKey().getZoneId(); + } + + return utf8Slice(Timestamps.formatTimestamp((int) precision, epochMicros, 0, zoneId, TIMESTAMP_FORMATTER)); + } + + @LiteralParameters({"x", "p"}) + @SqlType("varchar(x)") + public static Slice cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + ZoneId zoneId = ZoneOffset.UTC; + if (session.isLegacyTimestamp()) { + zoneId = session.getTimeZoneKey().getZoneId(); + } + + return utf8Slice(Timestamps.formatTimestamp((int) precision, timestamp.getEpochMicros(), timestamp.getPicosOfMicro(), zoneId, TIMESTAMP_FORMATTER)); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampWithTimezoneToTimestampCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampWithTimezoneToTimestampCast.java new file mode 100644 index 0000000000000..cb549a2e36bd4 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/TimestampWithTimezoneToTimestampCast.java @@ -0,0 +1,63 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import org.joda.time.chrono.ISOChronology; + +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.spi.type.DateTimeEncoding.unpackMillisUtc; +import static io.prestosql.spi.type.DateTimeEncoding.unpackZoneKey; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; +import static io.prestosql.util.DateTimeZoneIndex.getChronology; + +@ScalarOperator(CAST) +public final class TimestampWithTimezoneToTimestampCast +{ + private TimestampWithTimezoneToTimestampCast() {} + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static long cast(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long value) + { + long epochMillis; + if (session.isLegacyTimestamp()) { + epochMillis = unpackMillisUtc(value); + } + else { + ISOChronology chronology = getChronology(unpackZoneKey(value)); + epochMillis = chronology.getZone().convertUTCToLocal(unpackMillisUtc(value)); + } + + if (precision > 3) { + return scaleEpochMillisToMicros(epochMillis); + } + + return round(epochMillis, (int) (3 - precision)); + } + + @LiteralParameters("p") + @SqlType("timestamp(p)") + public static LongTimestamp cast(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long time) + { + return new LongTimestamp(cast(6, session, time), 0); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ToIso8601.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ToIso8601.java new file mode 100644 index 0000000000000..8694690e9d2fe --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ToIso8601.java @@ -0,0 +1,77 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.type.Constraint; + +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; + +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.type.Timestamps.formatTimestamp; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; + +@ScalarFunction("to_iso8601") +public final class ToIso8601 +{ + private static final DateTimeFormatter ISO8601_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd'T'HH:mm:ss"); + + // 1 digit for year sign + // 6 digits for year -- TODO: we should constrain this further. A 6-digit year seems useless + // 15 digits for -MM-DDTHH:MM:SS + // min(p, 1) for the fractional second period (i.e., no period if p == 0) + // p for the fractional digits + private static final String RESULT_LENGTH = "1 + 6 + 15 + min(p, 1) + p"; + + private ToIso8601() {} + + @LiteralParameters({"p", "n"}) + @SqlType("varchar(n)") + @Constraint(variable = "n", expression = RESULT_LENGTH) + public static Slice toIso8601(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp) + { + long epochMicros = timestamp; + if (precision <= 3) { + epochMicros = scaleEpochMillisToMicros(timestamp); + } + + ZoneId zoneId = ZoneOffset.UTC; + if (session.isLegacyTimestamp()) { + zoneId = session.getTimeZoneKey().getZoneId(); + } + + return utf8Slice(formatTimestamp((int) precision, epochMicros, 0, zoneId, ISO8601_FORMATTER)); + } + + @LiteralParameters({"p", "n"}) + @SqlType("varchar(n)") + @Constraint(variable = "n", expression = RESULT_LENGTH) + public static Slice toIso8601(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp) + { + ZoneId zoneId = ZoneOffset.UTC; + if (session.isLegacyTimestamp()) { + zoneId = session.getTimeZoneKey().getZoneId(); + } + + return utf8Slice(formatTimestamp((int) precision, timestamp.getEpochMicros(), timestamp.getPicosOfMicro(), zoneId, ISO8601_FORMATTER)); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ToUnixTime.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ToUnixTime.java new file mode 100644 index 0000000000000..e6b80e0378259 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/ToUnixTime.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; + +import static io.prestosql.type.Timestamps.MICROSECONDS_PER_SECOND; +import static io.prestosql.type.Timestamps.MILLISECONDS_PER_SECOND; +import static io.prestosql.type.Timestamps.PICOSECONDS_PER_SECOND; + +@ScalarFunction("to_unixtime") +public final class ToUnixTime +{ + private ToUnixTime() {} + + @LiteralParameters("p") + @SqlType(StandardTypes.DOUBLE) + public static double toUnixTime(@LiteralParameter("p") long precision, @SqlType("timestamp(p)") long timestamp) + { + if (precision <= 3) { + return timestamp * 1.0 / MILLISECONDS_PER_SECOND; + } + + return timestamp * 1.0 / MICROSECONDS_PER_SECOND; + } + + @LiteralParameters("p") + @SqlType(StandardTypes.DOUBLE) + public static double toUnixTime(@SqlType("timestamp(p)") LongTimestamp timestamp) + { + return timestamp.getEpochMicros() * 1.0 / MICROSECONDS_PER_SECOND + timestamp.getPicosOfMicro() * 1.0 / PICOSECONDS_PER_SECOND; + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/VarcharToTimestampCast.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/VarcharToTimestampCast.java new file mode 100644 index 0000000000000..d2282dbd96d9f --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/VarcharToTimestampCast.java @@ -0,0 +1,204 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarOperator; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.TimeZoneKey; +import io.prestosql.type.Timestamps; + +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.util.function.Function; +import java.util.regex.Matcher; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.prestosql.operator.scalar.StringFunctions.trim; +import static io.prestosql.spi.StandardErrorCode.INVALID_CAST_ARGUMENT; +import static io.prestosql.spi.function.OperatorType.CAST; +import static io.prestosql.spi.type.TimestampType.MAX_PRECISION; +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; +import static io.prestosql.type.Timestamps.MICROSECONDS_PER_SECOND; +import static io.prestosql.type.Timestamps.MILLISECONDS_PER_SECOND; +import static io.prestosql.type.Timestamps.longTimestamp; +import static io.prestosql.type.Timestamps.rescale; +import static io.prestosql.type.Timestamps.round; + +@ScalarOperator(CAST) +public final class VarcharToTimestampCast +{ + private VarcharToTimestampCast() {} + + @LiteralParameters({"x", "p"}) + @SqlType("timestamp(p)") + public static long castToShort(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("varchar(x)") Slice value) + { + // This accepts value with or without time zone + if (session.isLegacyTimestamp()) { + try { + return castToLegacyShortTimestamp((int) precision, session.getTimeZoneKey(), trim(value).toStringUtf8()); + } + catch (IllegalArgumentException e) { + throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); + } + } + + try { + return castToShortTimestamp((int) precision, trim(value).toStringUtf8(), timezone -> ZoneOffset.UTC); + } + catch (IllegalArgumentException e) { + throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); + } + } + + @LiteralParameters({"x", "p"}) + @SqlType("timestamp(p)") + public static LongTimestamp castToLong(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("varchar(x)") Slice value) + { + // This accepts value with or without time zone + if (session.isLegacyTimestamp()) { + try { + return castToLegacyLongTimestamp((int) precision, session.getTimeZoneKey(), trim(value).toStringUtf8()); + } + catch (IllegalArgumentException e) { + throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); + } + } + + try { + return castToLongTimestamp((int) precision, trim(value).toStringUtf8(), timezone -> ZoneOffset.UTC); + } + catch (IllegalArgumentException e) { + throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); + } + } + + public static long castToLegacyShortTimestamp(int precision, TimeZoneKey timeZoneKey, String value) + { + return castToShortTimestamp(precision, value, timezone -> { + if (timezone == null) { + return timeZoneKey.getZoneId(); + } + return ZoneId.of(timezone); + }); + } + + private static LongTimestamp castToLegacyLongTimestamp(int precision, TimeZoneKey timeZoneKey, String value) + { + return castToLongTimestamp(precision, value, timezone -> { + if (timezone == null) { + return timeZoneKey.getZoneId(); + } + return ZoneId.of(timezone); + }); + } + + private static long castToShortTimestamp(int precision, String value, Function zoneId) + { + checkArgument(precision <= MAX_SHORT_PRECISION, "precision must be less than max short timestamp precision"); + + Matcher matcher = Timestamps.DATETIME_PATTERN.matcher(value); + if (!matcher.matches()) { + throw new IllegalArgumentException("Invalid timestamp: " + value); + } + + String year = matcher.group("year"); + String month = matcher.group("month"); + String day = matcher.group("day"); + String hour = matcher.group("hour"); + String minute = matcher.group("minute"); + String second = matcher.group("second"); + String fraction = matcher.group("fraction"); + String timezone = matcher.group("timezone"); + + long epochSecond = ZonedDateTime.of( + Integer.parseInt(year), + Integer.parseInt(month), + Integer.parseInt(day), + hour == null ? 0 : Integer.parseInt(hour), + minute == null ? 0 : Integer.parseInt(minute), + second == null ? 0 : Integer.parseInt(second), + 0, + zoneId.apply(timezone)) + .toEpochSecond(); + + int actualPrecision = 0; + long fractionValue = 0; + if (fraction != null) { + actualPrecision = fraction.length(); + fractionValue = Long.parseLong(fraction); + } + + if (actualPrecision > precision) { + fractionValue = round(fractionValue, actualPrecision - precision); + } + + if (precision <= 3) { + return epochSecond * MILLISECONDS_PER_SECOND + rescale(fractionValue, actualPrecision, 3); + } + + // scale to micros + return epochSecond * MICROSECONDS_PER_SECOND + rescale(fractionValue, actualPrecision, 6); + } + + private static LongTimestamp castToLongTimestamp(int precision, String value, Function zoneId) + { + checkArgument(precision > MAX_SHORT_PRECISION && precision <= MAX_PRECISION, "precision out of range"); + + Matcher matcher = Timestamps.DATETIME_PATTERN.matcher(value); + if (!matcher.matches()) { + throw new IllegalArgumentException("Invalid timestamp: " + value); + } + + String year = matcher.group("year"); + String month = matcher.group("month"); + String day = matcher.group("day"); + String hour = matcher.group("hour"); + String minute = matcher.group("minute"); + String second = matcher.group("second"); + String fraction = matcher.group("fraction"); + String timezone = matcher.group("timezone"); + + long epochSecond = ZonedDateTime.of( + Integer.parseInt(year), + Integer.parseInt(month), + Integer.parseInt(day), + hour == null ? 0 : Integer.parseInt(hour), + minute == null ? 0 : Integer.parseInt(minute), + second == null ? 0 : Integer.parseInt(second), + 0, + zoneId.apply(timezone)) + .toEpochSecond(); + + int actualPrecision = 0; + long fractionValue = 0; + if (fraction != null) { + actualPrecision = fraction.length(); + fractionValue = Long.parseLong(fraction); + } + + if (actualPrecision > precision) { + fractionValue = round(fractionValue, actualPrecision - precision); + } + + return longTimestamp(epochSecond, rescale(fractionValue, actualPrecision, 12)); + } +} diff --git a/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/WithTimeZone.java b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/WithTimeZone.java new file mode 100644 index 0000000000000..a01e85c04903c --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/operator/scalar/timestamp/WithTimeZone.java @@ -0,0 +1,58 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.function.LiteralParameter; +import io.prestosql.spi.function.LiteralParameters; +import io.prestosql.spi.function.ScalarFunction; +import io.prestosql.spi.function.SqlType; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.StandardTypes; +import io.prestosql.spi.type.TimeZoneKey; +import org.joda.time.DateTimeZone; + +import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone; +import static io.prestosql.spi.type.TimeZoneKey.getTimeZoneKey; +import static io.prestosql.type.Timestamps.round; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; +import static io.prestosql.util.DateTimeZoneIndex.getDateTimeZone; + +@ScalarFunction("with_timezone") +public class WithTimeZone +{ + private WithTimeZone() {} + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) + public static long withTimezone(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") long timestamp, @SqlType("varchar(x)") Slice zoneId) + { + if (precision > 3) { + timestamp = scaleEpochMicrosToMillis(round(timestamp, 3)); + } + + TimeZoneKey toTimeZoneKey = getTimeZoneKey(zoneId.toStringUtf8()); + DateTimeZone fromDateTimeZone = session.isLegacyTimestamp() ? getDateTimeZone(session.getTimeZoneKey()) : DateTimeZone.UTC; + DateTimeZone toDateTimeZone = getDateTimeZone(toTimeZoneKey); + return packDateTimeWithZone(fromDateTimeZone.getMillisKeepLocal(toDateTimeZone, timestamp), toTimeZoneKey); + } + + @LiteralParameters({"x", "p"}) + @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) + public static long withTimezone(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("timestamp(p)") LongTimestamp timestamp, @SqlType("varchar(x)") Slice zoneId) + { + return withTimezone(6, session, timestamp.getEpochMicros(), zoneId); + } +} diff --git a/presto-main/src/main/java/io/prestosql/sql/analyzer/ExpressionAnalyzer.java b/presto-main/src/main/java/io/prestosql/sql/analyzer/ExpressionAnalyzer.java index b5116236e3541..29215a5758349 100644 --- a/presto-main/src/main/java/io/prestosql/sql/analyzer/ExpressionAnalyzer.java +++ b/presto-main/src/main/java/io/prestosql/sql/analyzer/ExpressionAnalyzer.java @@ -38,6 +38,7 @@ import io.prestosql.spi.type.DecimalParseResult; import io.prestosql.spi.type.Decimals; import io.prestosql.spi.type.RowType; +import io.prestosql.spi.type.TimestampType; import io.prestosql.spi.type.Type; import io.prestosql.spi.type.TypeNotFoundException; import io.prestosql.spi.type.TypeSignatureParameter; @@ -143,6 +144,7 @@ import static io.prestosql.spi.type.TimeType.TIME; import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TimestampType.createTimestampType; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.TinyintType.TINYINT; import static io.prestosql.spi.type.VarbinaryType.VARBINARY; @@ -155,18 +157,20 @@ import static io.prestosql.sql.analyzer.SemanticExceptions.semanticException; import static io.prestosql.sql.analyzer.TypeSignatureTranslator.toTypeSignature; import static io.prestosql.sql.tree.ArrayConstructor.ARRAY_CONSTRUCTOR; +import static io.prestosql.sql.tree.CurrentTime.Function.LOCALTIMESTAMP; import static io.prestosql.sql.tree.Extract.Field.TIMEZONE_HOUR; import static io.prestosql.sql.tree.Extract.Field.TIMEZONE_MINUTE; import static io.prestosql.type.ArrayParametricType.ARRAY; import static io.prestosql.type.IntervalDayTimeType.INTERVAL_DAY_TIME; import static io.prestosql.type.IntervalYearMonthType.INTERVAL_YEAR_MONTH; import static io.prestosql.type.JsonType.JSON; +import static io.prestosql.type.Timestamps.extractTimestampPrecision; +import static io.prestosql.type.Timestamps.parseLegacyTimestamp; +import static io.prestosql.type.Timestamps.parseTimestamp; +import static io.prestosql.type.Timestamps.timestampHasTimeZone; import static io.prestosql.type.UnknownType.UNKNOWN; -import static io.prestosql.util.DateTimeUtils.parseLegacyTimestamp; -import static io.prestosql.util.DateTimeUtils.parseTimestamp; import static io.prestosql.util.DateTimeUtils.parseTimestampWithTimeZone; import static io.prestosql.util.DateTimeUtils.timeHasTimeZone; -import static io.prestosql.util.DateTimeUtils.timestampHasTimeZone; import static java.lang.Math.toIntExact; import static java.lang.String.format; import static java.util.Collections.unmodifiableMap; @@ -366,7 +370,7 @@ protected Type visitRow(Row node, StackableAstVisitorContext context) @Override protected Type visitCurrentTime(CurrentTime node, StackableAstVisitorContext context) { - if (node.getPrecision() != null) { + if (node.getPrecision() != null && node.getFunction() != LOCALTIMESTAMP) { throw semanticException(NOT_SUPPORTED, node, "non-default precision not yet supported"); } @@ -385,7 +389,12 @@ protected Type visitCurrentTime(CurrentTime node, StackableAstVisitorContext { boolean withTimeZone = type.isWithTimeZone(); - if (type.getPrecision().isPresent()) { + if (type.getPrecision().isPresent() && type.getType() != DateTimeDataType.Type.TIMESTAMP) { throw new PrestoException(NOT_SUPPORTED, String.format("%s type with non-default precision not yet supported", type.getType())); } @@ -179,7 +179,24 @@ private static TypeSignature toTypeSignature(DateTimeDataType type, Set if (withTimeZone) { return TIMESTAMP_WITH_TIME_ZONE.getTypeSignature(); } - return TIMESTAMP.getTypeSignature(); + + List parameters = new ArrayList<>(); + + if (type.getPrecision().isPresent()) { + DataTypeParameter precision = type.getPrecision().get(); + if (precision instanceof NumericParameter) { + parameters.add(TypeSignatureParameter.numericParameter(Long.parseLong(((NumericParameter) precision).getValue()))); + } + else if (precision instanceof TypeParameter) { + DataType typeVariable = ((TypeParameter) precision).getValue(); + checkArgument(typeVariable instanceof GenericDataType && ((GenericDataType) typeVariable).getArguments().isEmpty()); + String variable = ((GenericDataType) typeVariable).getName().getValue(); + checkArgument(typeVariables.contains(variable), "Parameter to datetime type must be either a number or a type variable: %s", variable); + parameters.add(TypeSignatureParameter.typeVariable(variable)); + } + } + + return new TypeSignature(StandardTypes.TIMESTAMP, parameters); case TIME: if (withTimeZone) { return TIME_WITH_TIME_ZONE.getTypeSignature(); @@ -210,7 +227,15 @@ static DataType toDataType(TypeSignature typeSignature) case StandardTypes.TIMESTAMP_WITH_TIME_ZONE: return new DateTimeDataType(Optional.empty(), DateTimeDataType.Type.TIMESTAMP, true, Optional.empty()); case StandardTypes.TIMESTAMP: - return new DateTimeDataType(Optional.empty(), DateTimeDataType.Type.TIMESTAMP, false, Optional.empty()); + if (typeSignature.getParameters().isEmpty()) { + return new DateTimeDataType(Optional.empty(), DateTimeDataType.Type.TIMESTAMP, false, Optional.empty()); + } + + Optional argument = typeSignature.getParameters().stream() + .map(TypeSignatureTranslator::toTypeParameter) + .findAny(); + + return new DateTimeDataType(Optional.empty(), DateTimeDataType.Type.TIMESTAMP, false, argument); case StandardTypes.TIME_WITH_TIME_ZONE: return new DateTimeDataType(Optional.empty(), DateTimeDataType.Type.TIME, true, Optional.empty()); case StandardTypes.TIME: @@ -250,4 +275,16 @@ static DataType toDataType(TypeSignature typeSignature) .collect(toImmutableList())); } } + + private static DataTypeParameter toTypeParameter(TypeSignatureParameter parameter) + { + switch (parameter.getKind()) { + case LONG: + return new NumericParameter(Optional.empty(), String.valueOf(parameter.getLongLiteral())); + case TYPE: + return new TypeParameter(toDataType(parameter.getTypeSignature())); + default: + throw new UnsupportedOperationException("Unsupported parameter kind"); + } + } } diff --git a/presto-main/src/main/java/io/prestosql/sql/planner/DesugarAtTimeZoneRewriter.java b/presto-main/src/main/java/io/prestosql/sql/planner/DesugarAtTimeZoneRewriter.java index b88fb85292afd..c26efe82c282d 100644 --- a/presto-main/src/main/java/io/prestosql/sql/planner/DesugarAtTimeZoneRewriter.java +++ b/presto-main/src/main/java/io/prestosql/sql/planner/DesugarAtTimeZoneRewriter.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import io.prestosql.Session; import io.prestosql.metadata.Metadata; +import io.prestosql.spi.type.TimestampType; import io.prestosql.spi.type.Type; import io.prestosql.sql.tree.AtTimeZone; import io.prestosql.sql.tree.Cast; @@ -30,7 +31,6 @@ import static io.prestosql.spi.type.TimeType.TIME; import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.sql.analyzer.TypeSignatureTranslator.toSqlType; import static java.util.Objects.requireNonNull; @@ -79,7 +79,7 @@ public Expression rewriteAtTimeZone(AtTimeZone node, Void context, ExpressionTre valueType = TIME_WITH_TIME_ZONE; value = new Cast(value, toSqlType(valueType)); } - else if (valueType.equals(TIMESTAMP)) { + else if (valueType instanceof TimestampType) { valueType = TIMESTAMP_WITH_TIME_ZONE; value = new Cast(value, toSqlType(TIMESTAMP_WITH_TIME_ZONE)); } diff --git a/presto-main/src/main/java/io/prestosql/sql/planner/LiteralInterpreter.java b/presto-main/src/main/java/io/prestosql/sql/planner/LiteralInterpreter.java index a88d251689953..3528f2376d3cb 100644 --- a/presto-main/src/main/java/io/prestosql/sql/planner/LiteralInterpreter.java +++ b/presto-main/src/main/java/io/prestosql/sql/planner/LiteralInterpreter.java @@ -53,12 +53,12 @@ import static io.prestosql.sql.analyzer.SemanticExceptions.semanticException; import static io.prestosql.sql.analyzer.TypeSignatureProvider.fromTypes; import static io.prestosql.type.JsonType.JSON; +import static io.prestosql.type.Timestamps.parseLegacyTimestamp; +import static io.prestosql.type.Timestamps.parseTimestamp; import static io.prestosql.util.DateTimeUtils.parseDayTimeInterval; import static io.prestosql.util.DateTimeUtils.parseLegacyTime; -import static io.prestosql.util.DateTimeUtils.parseLegacyTimestamp; import static io.prestosql.util.DateTimeUtils.parseTimeWithTimeZone; import static io.prestosql.util.DateTimeUtils.parseTimeWithoutTimeZone; -import static io.prestosql.util.DateTimeUtils.parseTimestamp; import static io.prestosql.util.DateTimeUtils.parseTimestampWithTimeZone; import static io.prestosql.util.DateTimeUtils.parseYearMonthInterval; import static java.util.Objects.requireNonNull; @@ -181,15 +181,16 @@ else if (type instanceof TimeWithTimeZoneType) { } @Override - protected Long visitTimestampLiteral(TimestampLiteral node, ConnectorSession session) + protected Object visitTimestampLiteral(TimestampLiteral node, ConnectorSession session) { Type type = types.get(NodeRef.of(node)); if (type instanceof TimestampType) { + int precision = ((TimestampType) type).getPrecision(); if (session.isLegacyTimestamp()) { - return parseLegacyTimestamp(session.getTimeZoneKey(), node.getValue()); + return parseLegacyTimestamp(precision, session.getTimeZoneKey(), node.getValue()); } - return parseTimestamp(node.getValue()); + return parseTimestamp(precision, node.getValue()); } else if (type instanceof TimestampWithTimeZoneType) { return parseTimestampWithTimeZone(node.getValue()); diff --git a/presto-main/src/main/java/io/prestosql/sql/planner/iterative/rule/CanonicalizeExpressionRewriter.java b/presto-main/src/main/java/io/prestosql/sql/planner/iterative/rule/CanonicalizeExpressionRewriter.java index 1cc46ddecb413..e3af672eeb582 100644 --- a/presto-main/src/main/java/io/prestosql/sql/planner/iterative/rule/CanonicalizeExpressionRewriter.java +++ b/presto-main/src/main/java/io/prestosql/sql/planner/iterative/rule/CanonicalizeExpressionRewriter.java @@ -37,6 +37,7 @@ import io.prestosql.sql.tree.Literal; import io.prestosql.sql.tree.NodeRef; import io.prestosql.sql.tree.NotExpression; +import io.prestosql.sql.tree.NullLiteral; import io.prestosql.sql.tree.QualifiedName; import io.prestosql.sql.tree.Row; import io.prestosql.sql.tree.SearchedCaseExpression; @@ -51,6 +52,7 @@ import static io.prestosql.spi.type.VarcharType.VARCHAR; import static io.prestosql.sql.tree.ArithmeticBinaryExpression.Operator.ADD; import static io.prestosql.sql.tree.ArithmeticBinaryExpression.Operator.MULTIPLY; +import static io.prestosql.sql.tree.CurrentTime.Function.LOCALTIMESTAMP; import static java.util.Objects.requireNonNull; public final class CanonicalizeExpressionRewriter @@ -134,7 +136,7 @@ public Expression rewriteIfExpression(IfExpression node, Void context, Expressio @Override public Expression rewriteCurrentTime(CurrentTime node, Void context, ExpressionTreeRewriter treeRewriter) { - if (node.getPrecision() != null) { + if (node.getPrecision() != null && node.getFunction() != LOCALTIMESTAMP) { throw new UnsupportedOperationException("not yet implemented: non-default precision"); } @@ -157,7 +159,8 @@ public Expression rewriteCurrentTime(CurrentTime node, Void context, ExpressionT .build(); case LOCALTIMESTAMP: return new FunctionCallBuilder(metadata) - .setName(QualifiedName.of("localtimestamp")) + .setName(QualifiedName.of("$localtimestamp")) + .setArguments(ImmutableList.of(expressionTypes.get(NodeRef.of(node))), ImmutableList.of(new NullLiteral())) .build(); default: throw new UnsupportedOperationException("not yet implemented: " + node.getFunction()); diff --git a/presto-main/src/main/java/io/prestosql/sql/relational/SqlToRowExpressionTranslator.java b/presto-main/src/main/java/io/prestosql/sql/relational/SqlToRowExpressionTranslator.java index 717ad64319316..7eb19198af9fe 100644 --- a/presto-main/src/main/java/io/prestosql/sql/relational/SqlToRowExpressionTranslator.java +++ b/presto-main/src/main/java/io/prestosql/sql/relational/SqlToRowExpressionTranslator.java @@ -115,11 +115,11 @@ import static io.prestosql.sql.relational.SpecialForm.Form.SWITCH; import static io.prestosql.sql.relational.SpecialForm.Form.WHEN; import static io.prestosql.type.JsonType.JSON; +import static io.prestosql.type.Timestamps.parseLegacyTimestamp; +import static io.prestosql.type.Timestamps.parseTimestamp; import static io.prestosql.util.DateTimeUtils.parseDayTimeInterval; -import static io.prestosql.util.DateTimeUtils.parseLegacyTimestamp; import static io.prestosql.util.DateTimeUtils.parseTimeWithTimeZone; import static io.prestosql.util.DateTimeUtils.parseTimeWithoutTimeZone; -import static io.prestosql.util.DateTimeUtils.parseTimestamp; import static io.prestosql.util.DateTimeUtils.parseTimestampWithTimeZone; import static io.prestosql.util.DateTimeUtils.parseYearMonthInterval; import static java.util.Objects.requireNonNull; @@ -290,13 +290,14 @@ protected RowExpression visitTimestampLiteral(TimestampLiteral node, Void contex { Type type = getType(node); - long value; + Object value; if (type instanceof TimestampType) { + int precision = ((TimestampType) type).getPrecision(); if (isLegacyTimestamp) { - value = parseLegacyTimestamp(timeZoneKey, node.getValue()); + value = parseLegacyTimestamp(precision, timeZoneKey, node.getValue()); } else { - value = parseTimestamp(node.getValue()); + value = parseTimestamp(precision, node.getValue()); } } else if (type instanceof TimestampWithTimeZoneType) { diff --git a/presto-main/src/main/java/io/prestosql/testing/DateTimeTestingUtils.java b/presto-main/src/main/java/io/prestosql/testing/DateTimeTestingUtils.java index 4958ad38cd247..6b6183d506638 100644 --- a/presto-main/src/main/java/io/prestosql/testing/DateTimeTestingUtils.java +++ b/presto-main/src/main/java/io/prestosql/testing/DateTimeTestingUtils.java @@ -38,6 +38,7 @@ public final class DateTimeTestingUtils private DateTimeTestingUtils() {} public static SqlTimestamp sqlTimestampOf( + int precision, int year, int monthOfYear, int dayOfMonth, @@ -48,6 +49,7 @@ public static SqlTimestamp sqlTimestampOf( Session session) { return sqlTimestampOf( + precision, year, monthOfYear, dayOfMonth, @@ -61,6 +63,7 @@ public static SqlTimestamp sqlTimestampOf( } public static SqlTimestamp sqlTimestampOf( + int precision, int year, int monthOfYear, int dayOfMonth, @@ -73,36 +76,49 @@ public static SqlTimestamp sqlTimestampOf( ConnectorSession session) { if (session.isLegacyTimestamp()) { - return new SqlTimestamp(new DateTime(year, monthOfYear, dayOfMonth, hourOfDay, minuteOfHour, secondOfMinute, millisOfSecond, baseZone).getMillis(), timestampZone); + return SqlTimestamp.legacyFromMillis(precision, new DateTime(year, monthOfYear, dayOfMonth, hourOfDay, minuteOfHour, secondOfMinute, millisOfSecond, baseZone).getMillis(), timestampZone); } - return sqlTimestampOf(LocalDateTime.of(year, monthOfYear, dayOfMonth, hourOfDay, minuteOfHour, secondOfMinute, millisToNanos(millisOfSecond))); + return sqlTimestampOf(precision, LocalDateTime.of(year, monthOfYear, dayOfMonth, hourOfDay, minuteOfHour, secondOfMinute, millisToNanos(millisOfSecond))); } /** * Constructs standard (non-legacy) TIMESTAMP value corresponding to argument */ - public static SqlTimestamp sqlTimestampOf(LocalDateTime dateTime) + public static SqlTimestamp sqlTimestampOf(int precision, LocalDateTime dateTime) { - return new SqlTimestamp(DAYS.toMillis(dateTime.toLocalDate().toEpochDay()) + NANOSECONDS.toMillis(dateTime.toLocalTime().toNanoOfDay())); + return SqlTimestamp.fromMillis(precision, DAYS.toMillis(dateTime.toLocalDate().toEpochDay()) + NANOSECONDS.toMillis(dateTime.toLocalTime().toNanoOfDay())); } - public static SqlTimestamp sqlTimestampOf(DateTime dateTime, Session session) + public static SqlTimestamp sqlTimestampOf(int precision, DateTime dateTime, Session session) { - return sqlTimestampOf(dateTime, session.toConnectorSession()); + return sqlTimestampOf(precision, dateTime.getMillis(), session.toConnectorSession()); } - private static SqlTimestamp sqlTimestampOf(DateTime dateTime, ConnectorSession session) + /** + * @deprecated Use {@link #sqlTimestampOf(int precision, DateTime dateTime, Session session)} + */ + @Deprecated + public static SqlTimestamp sqlTimestampOf(DateTime dateTime, Session session) { - return sqlTimestampOf(dateTime.getMillis(), session); + return sqlTimestampOf(3, dateTime.getMillis(), session.toConnectorSession()); } + /** + * @deprecated Use {@link #sqlTimestampOf(int precision, long millis, ConnectorSession session)} + */ + @Deprecated public static SqlTimestamp sqlTimestampOf(long millis, ConnectorSession session) + { + return sqlTimestampOf(3, millis, session); + } + + public static SqlTimestamp sqlTimestampOf(int precision, long millis, ConnectorSession session) { if (session.isLegacyTimestamp()) { - return new SqlTimestamp(millis, session.getTimeZoneKey()); + return SqlTimestamp.legacyFromMillis(precision, millis, session.getTimeZoneKey()); } else { - return new SqlTimestamp(millis); + return SqlTimestamp.fromMillis(precision, millis); } } diff --git a/presto-main/src/main/java/io/prestosql/testing/MaterializedResult.java b/presto-main/src/main/java/io/prestosql/testing/MaterializedResult.java index 0ea1430c0130e..449a9c119970d 100644 --- a/presto-main/src/main/java/io/prestosql/testing/MaterializedResult.java +++ b/presto-main/src/main/java/io/prestosql/testing/MaterializedResult.java @@ -36,6 +36,7 @@ import io.prestosql.spi.type.SqlTimestamp; import io.prestosql.spi.type.SqlTimestampWithTimeZone; import io.prestosql.spi.type.TimeZoneKey; +import io.prestosql.spi.type.TimestampType; import io.prestosql.spi.type.Type; import io.prestosql.spi.type.VarcharType; @@ -74,11 +75,11 @@ import static io.prestosql.spi.type.SmallintType.SMALLINT; import static io.prestosql.spi.type.TimeType.TIME; import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.TinyintType.TINYINT; import static io.prestosql.spi.type.VarbinaryType.VARBINARY; import static io.prestosql.type.JsonType.JSON; +import static io.prestosql.type.Timestamps.scaleEpochMicrosToMillis; import static java.lang.Float.floatToRawIntBits; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -299,9 +300,15 @@ else if (TIME_WITH_TIME_ZONE.equals(type)) { TimeZoneKey timeZoneKey = ((SqlTimeWithTimeZone) value).getTimeZoneKey(); type.writeLong(blockBuilder, packDateTimeWithZone(millisUtc, timeZoneKey)); } - else if (TIMESTAMP.equals(type)) { - long millisUtc = ((SqlTimestamp) value).getMillisUtc(); - type.writeLong(blockBuilder, millisUtc); + else if (type instanceof TimestampType) { + long micros = ((SqlTimestamp) value).getEpochMicros(); + int precision = ((TimestampType) type).getPrecision(); + if (precision <= 3) { + type.writeLong(blockBuilder, scaleEpochMicrosToMillis(micros)); + } + else { + type.writeLong(blockBuilder, micros); + } } else if (TIMESTAMP_WITH_TIME_ZONE.equals(type)) { long millisUtc = ((SqlTimestampWithTimeZone) value).getMillisUtc(); diff --git a/presto-main/src/main/java/io/prestosql/type/DateOperators.java b/presto-main/src/main/java/io/prestosql/type/DateOperators.java index c05c9269d639c..2b58816e22b93 100644 --- a/presto-main/src/main/java/io/prestosql/type/DateOperators.java +++ b/presto-main/src/main/java/io/prestosql/type/DateOperators.java @@ -100,21 +100,6 @@ public static boolean greaterThanOrEqual(@SqlType(StandardTypes.DATE) long left, return left >= right; } - @ScalarOperator(CAST) - @SqlType(StandardTypes.TIMESTAMP) - public static long castToTimestamp(ConnectorSession session, @SqlType(StandardTypes.DATE) long value) - { - if (session.isLegacyTimestamp()) { - long utcMillis = TimeUnit.DAYS.toMillis(value); - - // date is encoded as milliseconds at midnight in UTC - // convert to midnight in the session timezone - ISOChronology chronology = getChronology(session.getTimeZoneKey()); - return utcMillis - chronology.getZone().getOffset(utcMillis); - } - return TimeUnit.DAYS.toMillis(value); - } - @ScalarOperator(CAST) @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) public static long castToTimestampWithTimeZone(ConnectorSession session, @SqlType(StandardTypes.DATE) long value) diff --git a/presto-main/src/main/java/io/prestosql/type/DateTimeOperators.java b/presto-main/src/main/java/io/prestosql/type/DateTimeOperators.java index ae7020abfcb8a..13fe1e6f242ca 100644 --- a/presto-main/src/main/java/io/prestosql/type/DateTimeOperators.java +++ b/presto-main/src/main/java/io/prestosql/type/DateTimeOperators.java @@ -85,20 +85,6 @@ public static long intervalDayToSecondPlusTimeWithTimeZone(@SqlType(StandardType return timeWithTimeZonePlusIntervalDayToSecond(time, interval); } - @ScalarOperator(ADD) - @SqlType(StandardTypes.TIMESTAMP) - public static long timestampPlusIntervalDayToSecond(@SqlType(StandardTypes.TIMESTAMP) long timestamp, @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) - { - return timestamp + interval; - } - - @ScalarOperator(ADD) - @SqlType(StandardTypes.TIMESTAMP) - public static long intervalDayToSecondPlusTimestamp(@SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - return timestampPlusIntervalDayToSecond(timestamp, interval); - } - @ScalarOperator(ADD) @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) public static long timestampWithTimeZonePlusIntervalDayToSecond(@SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long timestamp, @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) @@ -156,23 +142,6 @@ public static long intervalYearToMonthPlusTimeWithTimeZone(@SqlType(StandardType return time; } - @ScalarOperator(ADD) - @SqlType(StandardTypes.TIMESTAMP) - public static long timestampPlusIntervalYearToMonth(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp, @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) - { - if (session.isLegacyTimestamp()) { - return getChronology(session.getTimeZoneKey()).monthOfYear().add(timestamp, interval); - } - return MONTH_OF_YEAR_UTC.add(timestamp, interval); - } - - @ScalarOperator(ADD) - @SqlType(StandardTypes.TIMESTAMP) - public static long intervalYearToMonthPlusTimestamp(ConnectorSession session, @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval, @SqlType(StandardTypes.TIMESTAMP) long timestamp) - { - return timestampPlusIntervalYearToMonth(session, timestamp, interval); - } - @ScalarOperator(ADD) @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) public static long timestampWithTimeZonePlusIntervalYearToMonth(@SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long timestamp, @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) @@ -211,13 +180,6 @@ public static long timeWithTimeZoneMinusIntervalDayToSecond(@SqlType(StandardTyp return timeWithTimeZonePlusIntervalDayToSecond(time, -interval); } - @ScalarOperator(SUBTRACT) - @SqlType(StandardTypes.TIMESTAMP) - public static long timestampMinusIntervalDayToSecond(@SqlType(StandardTypes.TIMESTAMP) long timestamp, @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) - { - return timestamp - interval; - } - @ScalarOperator(SUBTRACT) @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) public static long timestampWithTimeZoneMinusIntervalDayToSecond(@SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long timestamp, @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) long interval) @@ -246,13 +208,6 @@ public static long timeWithTimeZoneMinusIntervalYearToMonth(@SqlType(StandardTyp return time; } - @ScalarOperator(SUBTRACT) - @SqlType(StandardTypes.TIMESTAMP) - public static long timestampMinusIntervalYearToMonth(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long timestamp, @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) - { - return timestampPlusIntervalYearToMonth(session, timestamp, -interval); - } - @ScalarOperator(SUBTRACT) @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) public static long timestampWithTimeZoneMinusIntervalYearToMonth(@SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long timestamp, @SqlType(StandardTypes.INTERVAL_YEAR_TO_MONTH) long interval) diff --git a/presto-main/src/main/java/io/prestosql/type/TimeOperators.java b/presto-main/src/main/java/io/prestosql/type/TimeOperators.java index 6a0ac7d327668..8fb1d103ee6c4 100644 --- a/presto-main/src/main/java/io/prestosql/type/TimeOperators.java +++ b/presto-main/src/main/java/io/prestosql/type/TimeOperators.java @@ -122,13 +122,6 @@ public static long castToTimeWithTimeZone(ConnectorSession session, @SqlType(Sta return packDateTimeWithZone(localChronology.getZone().convertLocalToUTC(value, false), session.getTimeZoneKey()); } - @ScalarOperator(CAST) - @SqlType(StandardTypes.TIMESTAMP) - public static long castToTimestamp(@SqlType(StandardTypes.TIME) long value) - { - return value; - } - @ScalarOperator(CAST) @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) public static long castToTimestampWithTimeZone(ConnectorSession session, @SqlType(StandardTypes.TIME) long value) diff --git a/presto-main/src/main/java/io/prestosql/type/TimeWithTimeZoneOperators.java b/presto-main/src/main/java/io/prestosql/type/TimeWithTimeZoneOperators.java index 3c47b9709ab5c..0ffdf827e2b60 100644 --- a/presto-main/src/main/java/io/prestosql/type/TimeWithTimeZoneOperators.java +++ b/presto-main/src/main/java/io/prestosql/type/TimeWithTimeZoneOperators.java @@ -15,6 +15,7 @@ import io.airlift.slice.Slice; import io.airlift.slice.XxHash64; +import io.prestosql.operator.scalar.timestamp.TimeWithTimezoneToTimestampCast; import io.prestosql.spi.block.Block; import io.prestosql.spi.connector.ConnectorSession; import io.prestosql.spi.function.BlockIndex; @@ -26,11 +27,6 @@ import io.prestosql.spi.function.SqlType; import io.prestosql.spi.type.AbstractLongType; import io.prestosql.spi.type.StandardTypes; -import org.joda.time.chrono.ISOChronology; - -import java.time.Instant; -import java.time.ZoneOffset; -import java.time.temporal.ChronoField; import static io.airlift.slice.Slices.utf8Slice; import static io.prestosql.spi.function.OperatorType.CAST; @@ -46,15 +42,13 @@ import static io.prestosql.spi.function.OperatorType.SUBTRACT; import static io.prestosql.spi.function.OperatorType.XX_HASH_64; import static io.prestosql.spi.type.DateTimeEncoding.unpackMillisUtc; -import static io.prestosql.spi.type.DateTimeEncoding.unpackZoneKey; import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; import static io.prestosql.util.DateTimeUtils.parseTimeWithTimeZone; import static io.prestosql.util.DateTimeUtils.printTimeWithTimeZone; -import static io.prestosql.util.DateTimeZoneIndex.getChronology; public final class TimeWithTimeZoneOperators { - private static final long REFERENCE_TIMESTAMP_UTC = System.currentTimeMillis(); + public static final long REFERENCE_TIMESTAMP_UTC = System.currentTimeMillis(); private TimeWithTimeZoneOperators() { @@ -117,24 +111,7 @@ public static long castToTime(ConnectorSession session, @SqlType(StandardTypes.T { // This is exactly the same operation as for TIME WITH TIME ZONE -> TIMESTAMP, as the representations // of those types are aligned in range that is covered by TIME WITH TIME ZONE. - return castToTimestamp(session, value); - } - - @ScalarOperator(CAST) - @SqlType(StandardTypes.TIMESTAMP) - public static long castToTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) long value) - { - if (session.isLegacyTimestamp()) { - return unpackMillisUtc(value); - } - - // This is hack that we need to use as the timezone interpretation depends on date (not only on time) - // TODO remove REFERENCE_TIMESTAMP_UTC when removing support for political time zones in TIME WIT TIME ZONE - long currentMillisOfDay = ChronoField.MILLI_OF_DAY.getFrom(Instant.ofEpochMilli(REFERENCE_TIMESTAMP_UTC).atZone(ZoneOffset.UTC)); - long timeMillisUtcInCurrentDay = REFERENCE_TIMESTAMP_UTC - currentMillisOfDay + unpackMillisUtc(value); - - ISOChronology chronology = getChronology(unpackZoneKey(value)); - return unpackMillisUtc(value) + chronology.getZone().getOffset(timeMillisUtcInCurrentDay); + return TimeWithTimezoneToTimestampCast.cast(3, session, value); } @ScalarOperator(CAST) diff --git a/presto-main/src/main/java/io/prestosql/type/TimestampOperators.java b/presto-main/src/main/java/io/prestosql/type/TimestampOperators.java deleted file mode 100644 index 0cf7656b72cb1..0000000000000 --- a/presto-main/src/main/java/io/prestosql/type/TimestampOperators.java +++ /dev/null @@ -1,263 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.prestosql.type; - -import io.airlift.slice.Slice; -import io.airlift.slice.XxHash64; -import io.prestosql.spi.PrestoException; -import io.prestosql.spi.block.Block; -import io.prestosql.spi.connector.ConnectorSession; -import io.prestosql.spi.function.BlockIndex; -import io.prestosql.spi.function.BlockPosition; -import io.prestosql.spi.function.IsNull; -import io.prestosql.spi.function.LiteralParameters; -import io.prestosql.spi.function.ScalarFunction; -import io.prestosql.spi.function.ScalarOperator; -import io.prestosql.spi.function.SqlNullable; -import io.prestosql.spi.function.SqlType; -import io.prestosql.spi.type.AbstractLongType; -import io.prestosql.spi.type.StandardTypes; -import org.joda.time.chrono.ISOChronology; - -import java.util.concurrent.TimeUnit; - -import static io.airlift.slice.SliceUtf8.trim; -import static io.airlift.slice.Slices.utf8Slice; -import static io.prestosql.spi.StandardErrorCode.INVALID_CAST_ARGUMENT; -import static io.prestosql.spi.function.OperatorType.CAST; -import static io.prestosql.spi.function.OperatorType.EQUAL; -import static io.prestosql.spi.function.OperatorType.GREATER_THAN; -import static io.prestosql.spi.function.OperatorType.GREATER_THAN_OR_EQUAL; -import static io.prestosql.spi.function.OperatorType.HASH_CODE; -import static io.prestosql.spi.function.OperatorType.INDETERMINATE; -import static io.prestosql.spi.function.OperatorType.IS_DISTINCT_FROM; -import static io.prestosql.spi.function.OperatorType.LESS_THAN; -import static io.prestosql.spi.function.OperatorType.LESS_THAN_OR_EQUAL; -import static io.prestosql.spi.function.OperatorType.NOT_EQUAL; -import static io.prestosql.spi.function.OperatorType.SUBTRACT; -import static io.prestosql.spi.function.OperatorType.XX_HASH_64; -import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; -import static io.prestosql.type.DateTimeOperators.modulo24Hour; -import static io.prestosql.util.DateTimeUtils.convertToLegacyTimestamp; -import static io.prestosql.util.DateTimeUtils.convertToTimestamp; -import static io.prestosql.util.DateTimeUtils.printTimestampWithoutTimeZone; -import static io.prestosql.util.DateTimeZoneIndex.getChronology; - -public final class TimestampOperators -{ - private TimestampOperators() {} - - @ScalarOperator(SUBTRACT) - @SqlType(StandardTypes.INTERVAL_DAY_TO_SECOND) - public static long subtract(@SqlType(StandardTypes.TIMESTAMP) long left, @SqlType(StandardTypes.TIMESTAMP) long right) - { - return left - right; - } - - @ScalarOperator(EQUAL) - @SqlType(StandardTypes.BOOLEAN) - @SqlNullable - public static Boolean equal(@SqlType(StandardTypes.TIMESTAMP) long left, @SqlType(StandardTypes.TIMESTAMP) long right) - { - return left == right; - } - - @ScalarOperator(NOT_EQUAL) - @SqlType(StandardTypes.BOOLEAN) - @SqlNullable - public static Boolean notEqual(@SqlType(StandardTypes.TIMESTAMP) long left, @SqlType(StandardTypes.TIMESTAMP) long right) - { - return left != right; - } - - @ScalarOperator(LESS_THAN) - @SqlType(StandardTypes.BOOLEAN) - public static boolean lessThan(@SqlType(StandardTypes.TIMESTAMP) long left, @SqlType(StandardTypes.TIMESTAMP) long right) - { - return left < right; - } - - @ScalarOperator(LESS_THAN_OR_EQUAL) - @SqlType(StandardTypes.BOOLEAN) - public static boolean lessThanOrEqual(@SqlType(StandardTypes.TIMESTAMP) long left, @SqlType(StandardTypes.TIMESTAMP) long right) - { - return left <= right; - } - - @ScalarOperator(GREATER_THAN) - @SqlType(StandardTypes.BOOLEAN) - public static boolean greaterThan(@SqlType(StandardTypes.TIMESTAMP) long left, @SqlType(StandardTypes.TIMESTAMP) long right) - { - return left > right; - } - - @ScalarOperator(GREATER_THAN_OR_EQUAL) - @SqlType(StandardTypes.BOOLEAN) - public static boolean greaterThanOrEqual(@SqlType(StandardTypes.TIMESTAMP) long left, @SqlType(StandardTypes.TIMESTAMP) long right) - { - return left >= right; - } - - @ScalarFunction("date") - @ScalarOperator(CAST) - @SqlType(StandardTypes.DATE) - public static long castToDate(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long value) - { - ISOChronology chronology; - if (session.isLegacyTimestamp()) { - // round down the current timestamp to days - chronology = getChronology(session.getTimeZoneKey()); - long date = chronology.dayOfYear().roundFloor(value); - // date is currently midnight in timezone of the session - // convert to UTC - long millis = date + chronology.getZone().getOffset(date); - return TimeUnit.MILLISECONDS.toDays(millis); - } - - return TimeUnit.MILLISECONDS.toDays(value); - } - - @ScalarOperator(CAST) - @SqlType(StandardTypes.TIME) - public static long castToTime(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long value) - { - if (session.isLegacyTimestamp()) { - return modulo24Hour(getChronology(session.getTimeZoneKey()), value); - } - - return modulo24Hour(value); - } - - @ScalarOperator(CAST) - @SqlType(StandardTypes.TIME_WITH_TIME_ZONE) - public static long castToTimeWithTimeZone(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long value) - { - if (session.isLegacyTimestamp()) { - int timeMillis = modulo24Hour(getChronology(session.getTimeZoneKey()), value); - return packDateTimeWithZone(timeMillis, session.getTimeZoneKey()); - } - - ISOChronology localChronology = getChronology(session.getTimeZoneKey()); - // This cast does treat TIMESTAMP as wall time in session TZ. This means that in order to get - // its UTC representation we need to shift the value by the offset of TZ. - return packDateTimeWithZone(localChronology.getZone().convertLocalToUTC(modulo24Hour(value), false), session.getTimeZoneKey()); - } - - @ScalarOperator(CAST) - @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) - public static long castToTimestampWithTimeZone(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long value) - { - if (session.isLegacyTimestamp()) { - return packDateTimeWithZone(value, session.getTimeZoneKey()); - } - - ISOChronology localChronology = getChronology(session.getTimeZoneKey()); - // This cast does treat TIMESTAMP as wall time in session TZ. This means that in order to get - // its UTC representation we need to shift the value by the offset of TZ. - return packDateTimeWithZone(localChronology.getZone().convertLocalToUTC(value, false), session.getTimeZoneKey()); - } - - @ScalarOperator(CAST) - @LiteralParameters("x") - @SqlType("varchar(x)") - public static Slice castToSlice(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP) long value) - { - if (session.isLegacyTimestamp()) { - return utf8Slice(printTimestampWithoutTimeZone(session.getTimeZoneKey(), value)); - } - - return utf8Slice(printTimestampWithoutTimeZone(value)); - } - - @ScalarOperator(CAST) - @LiteralParameters("x") - @SqlType(StandardTypes.TIMESTAMP) - public static long castFromSlice(ConnectorSession session, @SqlType("varchar(x)") Slice value) - { - // This accepts value with or without time zone - if (session.isLegacyTimestamp()) { - try { - return convertToLegacyTimestamp(session.getTimeZoneKey(), trim(value).toStringUtf8()); - } - catch (IllegalArgumentException e) { - throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); - } - } - - try { - return convertToTimestamp(trim(value).toStringUtf8()); - } - catch (IllegalArgumentException e) { - throw new PrestoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); - } - } - - @ScalarOperator(HASH_CODE) - @SqlType(StandardTypes.BIGINT) - public static long hashCode(@SqlType(StandardTypes.TIMESTAMP) long value) - { - return AbstractLongType.hash(value); - } - - @ScalarOperator(IS_DISTINCT_FROM) - public static final class TimestampDistinctFromOperator - { - @SqlType(StandardTypes.BOOLEAN) - public static boolean isDistinctFrom( - @SqlType(StandardTypes.TIMESTAMP) long left, - @IsNull boolean leftNull, - @SqlType(StandardTypes.TIMESTAMP) long right, - @IsNull boolean rightNull) - { - if (leftNull != rightNull) { - return true; - } - if (leftNull) { - return false; - } - return notEqual(left, right); - } - - @SqlType(StandardTypes.BOOLEAN) - public static boolean isDistinctFrom( - @BlockPosition @SqlType(value = StandardTypes.TIMESTAMP, nativeContainerType = long.class) Block left, - @BlockIndex int leftPosition, - @BlockPosition @SqlType(value = StandardTypes.TIMESTAMP, nativeContainerType = long.class) Block right, - @BlockIndex int rightPosition) - { - if (left.isNull(leftPosition) != right.isNull(rightPosition)) { - return true; - } - if (left.isNull(leftPosition)) { - return false; - } - return notEqual(TIMESTAMP.getLong(left, leftPosition), TIMESTAMP.getLong(right, rightPosition)); - } - } - - @ScalarOperator(INDETERMINATE) - @SqlType(StandardTypes.BOOLEAN) - public static boolean indeterminate(@SqlType(StandardTypes.TIMESTAMP) long value, @IsNull boolean isNull) - { - return isNull; - } - - @ScalarOperator(XX_HASH_64) - @SqlType(StandardTypes.BIGINT) - public static long xxHash64(@SqlType(StandardTypes.TIMESTAMP) long value) - { - return XxHash64.hash(value); - } -} diff --git a/presto-main/src/main/java/io/prestosql/type/TimestampWithTimeZoneOperators.java b/presto-main/src/main/java/io/prestosql/type/TimestampWithTimeZoneOperators.java index 46052e1a54b80..b6c116778c736 100644 --- a/presto-main/src/main/java/io/prestosql/type/TimestampWithTimeZoneOperators.java +++ b/presto-main/src/main/java/io/prestosql/type/TimestampWithTimeZoneOperators.java @@ -15,6 +15,7 @@ import io.airlift.slice.Slice; import io.airlift.slice.XxHash64; +import io.prestosql.operator.scalar.timestamp.TimestampWithTimezoneToTimestampCast; import io.prestosql.spi.PrestoException; import io.prestosql.spi.block.Block; import io.prestosql.spi.connector.ConnectorSession; @@ -54,7 +55,6 @@ import static io.prestosql.type.DateTimeOperators.modulo24Hour; import static io.prestosql.util.DateTimeUtils.convertToTimestampWithTimeZone; import static io.prestosql.util.DateTimeUtils.printTimestampWithTimeZone; -import static io.prestosql.util.DateTimeZoneIndex.getChronology; import static io.prestosql.util.DateTimeZoneIndex.unpackChronology; public final class TimestampWithTimeZoneOperators @@ -134,7 +134,7 @@ public static long castToTime(ConnectorSession session, @SqlType(StandardTypes.T return modulo24Hour(unpackChronology(value), unpackMillisUtc(value)); } - return modulo24Hour(castToTimestamp(session, value)); + return modulo24Hour(TimestampWithTimezoneToTimestampCast.cast(3, session, value)); } @ScalarOperator(CAST) @@ -146,7 +146,7 @@ public static long castToTimeWithTimeZone(ConnectorSession session, @SqlType(Sta return packDateTimeWithZone(millis, unpackZoneKey(value)); } - long millis = modulo24Hour(castToTimestamp(session, value)); + long millis = modulo24Hour(TimestampWithTimezoneToTimestampCast.cast(3, session, value)); ISOChronology localChronology = unpackChronology(value); // This cast does treat TIME as wall time in given TZ. This means that in order to get // its UTC representation we need to shift the value by the offset of TZ. @@ -156,18 +156,6 @@ public static long castToTimeWithTimeZone(ConnectorSession session, @SqlType(Sta return packDateTimeWithZone(millis - localChronology.getZone().getOffset(millis), unpackZoneKey(value)); } - @ScalarOperator(CAST) - @SqlType(StandardTypes.TIMESTAMP) - public static long castToTimestamp(ConnectorSession session, @SqlType(StandardTypes.TIMESTAMP_WITH_TIME_ZONE) long value) - { - if (session.isLegacyTimestamp()) { - return unpackMillisUtc(value); - } - - ISOChronology chronology = getChronology(unpackZoneKey(value)); - return chronology.getZone().convertUTCToLocal(unpackMillisUtc(value)); - } - @ScalarOperator(CAST) @LiteralParameters("x") @SqlType("varchar(x)") diff --git a/presto-main/src/main/java/io/prestosql/type/Timestamps.java b/presto-main/src/main/java/io/prestosql/type/Timestamps.java new file mode 100644 index 0000000000000..8ad92b07cbdfb --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/type/Timestamps.java @@ -0,0 +1,352 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.type; + +import io.prestosql.spi.block.Block; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.LongTimestamp; +import io.prestosql.spi.type.TimeZoneKey; +import io.prestosql.spi.type.TimestampType; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.prestosql.spi.type.TimestampType.MAX_PRECISION; +import static io.prestosql.spi.type.TimestampType.MAX_SHORT_PRECISION; +import static java.lang.Math.floorMod; +import static java.lang.Math.multiplyExact; +import static java.lang.String.format; +import static java.time.temporal.ChronoField.MICRO_OF_SECOND; + +public final class Timestamps +{ + public static final Pattern DATETIME_PATTERN = Pattern.compile("" + + "(?\\d\\d\\d\\d)-(?\\d{1,2})-(?\\d{1,2})" + + "(?: (?\\d{1,2}):(?\\d{1,2})(?::(?\\d{1,2})(?:\\.(?\\d+))?)?)?" + + "\\s*(?.+)?"); + private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss"); + + private static final long[] POWERS_OF_TEN = { + 1L, + 10L, + 100L, + 1000L, + 10_000L, + 100_000L, + 1_000_000L, + 10_000_000L, + 100_000_000L, + 1_000_000_000L, + 10_000_000_000L, + 100_000_000_000L, + 1000_000_000_000L + }; + + public static final int MILLISECONDS_PER_SECOND = 1000; + public static final int MICROSECONDS_PER_SECOND = 1_000_000; + public static final int MICROSECONDS_PER_MILLISECOND = 1000; + public static final long PICOSECONDS_PER_SECOND = 1_000_000_000_000L; + public static final int NANOSECONDS_PER_MICROSECOND = 1_000; + public static final int PICOSECONDS_PER_MICROSECOND = 1_000_000; + public static final int PICOSECONDS_PER_NANOSECOND = 1000; + + private Timestamps() {} + + private static long roundDiv(long value, long factor) + { + checkArgument(factor > 0, "factor must be positive"); + + if (value >= 0) { + return (value + (factor / 2)) / factor; + } + + return (value + 1 - (factor / 2)) / factor; + } + + public static long scaleEpochMicrosToMillis(long value) + { + return Math.floorDiv(value, MICROSECONDS_PER_MILLISECOND); + } + + private static long scaleEpochMicrosToSeconds(long epochMicros) + { + return Math.floorDiv(epochMicros, MICROSECONDS_PER_SECOND); + } + + public static long scaleEpochMillisToMicros(long epochMillis) + { + return multiplyExact(epochMillis, MICROSECONDS_PER_MILLISECOND); + } + + public static long epochSecondToMicrosWithRounding(long epochSecond, long picoOfSecond) + { + return epochSecond * MICROSECONDS_PER_SECOND + roundDiv(picoOfSecond, PICOSECONDS_PER_MICROSECOND); + } + + public static int getMicrosOfSecond(long epochMicros) + { + return floorMod(epochMicros, MICROSECONDS_PER_SECOND); + } + + public static int getMicrosOfMilli(long epochMicros) + { + return floorMod(epochMicros, MICROSECONDS_PER_MILLISECOND); + } + + public static long round(long value, int magnitude) + { + return roundToNearest(value, POWERS_OF_TEN[magnitude]); + } + + public static long roundToNearest(long value, long bound) + { + return roundDiv(value, bound) * bound; + } + + private static long scaleFactor(int fromPrecision, int toPrecision) + { + if (fromPrecision > toPrecision) { + throw new IllegalArgumentException("fromPrecision must be <= toPrecision"); + } + + return POWERS_OF_TEN[toPrecision - fromPrecision]; + } + + /** + * Rescales a value of the given precision to another precision by adding 0s or truncating. + */ + public static long rescale(long value, int fromPrecision, int toPrecision) + { + if (value < 0) { + throw new IllegalArgumentException("value must be >= 0"); + } + + if (fromPrecision <= toPrecision) { + value *= scaleFactor(fromPrecision, toPrecision); + } + else { + value /= scaleFactor(toPrecision, fromPrecision); + } + + return value; + } + + public static boolean timestampHasTimeZone(String value) + { + Matcher matcher = DATETIME_PATTERN.matcher(value); + if (!matcher.matches()) { + throw new IllegalArgumentException(format("Invalid timestamp '%s'", value)); + } + + return matcher.group("timezone") != null; + } + + public static int extractTimestampPrecision(String value) + { + Matcher matcher = DATETIME_PATTERN.matcher(value); + if (!matcher.matches()) { + throw new IllegalArgumentException(format("Invalid timestamp '%s'", value)); + } + + String fraction = matcher.group("fraction"); + if (fraction == null) { + return 0; + } + + return fraction.length(); + } + + public static LocalDateTime toLocalDateTime(TimestampType type, ConnectorSession session, Block block, int position) + { + int precision = type.getPrecision(); + + long epochMicros; + int picosOfMicro = 0; + if (precision <= 3) { + epochMicros = scaleEpochMillisToMicros(type.getLong(block, position)); + } + else if (precision <= MAX_SHORT_PRECISION) { + epochMicros = type.getLong(block, position); + } + else { + LongTimestamp timestamp = (LongTimestamp) type.getObject(block, position); + epochMicros = timestamp.getEpochMicros(); + picosOfMicro = timestamp.getPicosOfMicro(); + } + + long epochSecond = scaleEpochMicrosToSeconds(epochMicros); + int nanoFraction = getMicrosOfSecond(epochMicros) * NANOSECONDS_PER_MICROSECOND + (int) (roundToNearest(picosOfMicro, PICOSECONDS_PER_NANOSECOND) / PICOSECONDS_PER_NANOSECOND); + + Instant instant = Instant.ofEpochSecond(epochSecond, nanoFraction); + if (session.isLegacyTimestamp()) { + return LocalDateTime.ofInstant(instant, session.getTimeZoneKey().getZoneId()); + } + + return LocalDateTime.ofInstant(instant, ZoneOffset.UTC); + } + + /** + * Formats a timestamp of the given precision. This method doesn't do any rounding, so it's expected that the + * combination of [epochMicros, picosSecond] is already rounded to the provided precision if necessary + */ + public static String formatTimestamp(int precision, long epochMicros, int picosOfMicro, ZoneId zoneId) + { + return formatTimestamp(precision, epochMicros, picosOfMicro, zoneId, TIMESTAMP_FORMATTER); + } + + /** + * Formats a timestamp of the given precision. This method doesn't do any rounding, so it's expected that the + * combination of [epochMicros, picosSecond] is already rounded to the provided precision if necessary + */ + public static String formatTimestamp(int precision, long epochMicros, int picosOfMicro, ZoneId zoneId, DateTimeFormatter yearToSecondFormatter) + { + checkArgument(picosOfMicro >= 0 && picosOfMicro < PICOSECONDS_PER_MICROSECOND, "picosOfMicro is out of range [0, 1_000_000]"); + + Instant instant = Instant.ofEpochSecond(scaleEpochMicrosToSeconds(epochMicros)); + LocalDateTime dateTime = LocalDateTime.ofInstant(instant, zoneId); + + StringBuilder builder = new StringBuilder(); + builder.append(yearToSecondFormatter.format(dateTime)); + if (precision > 0) { + long picoFraction = ((long) getMicrosOfSecond(epochMicros)) * PICOSECONDS_PER_MICROSECOND + picosOfMicro; + builder.append("."); + builder.append(format("%0" + precision + "d", rescale(picoFraction, 12, precision))); + } + + return builder.toString(); + } + + public static Object parseTimestamp(int precision, String value) + { + if (precision <= MAX_SHORT_PRECISION) { + return parseShortTimestamp(value, ZoneOffset.UTC); + } + + return parseLongTimestamp(value, ZoneOffset.UTC); + } + + public static Object parseLegacyTimestamp(int precision, TimeZoneKey timeZoneKey, String value) + { + if (precision <= MAX_SHORT_PRECISION) { + return parseShortTimestamp(value, timeZoneKey.getZoneId()); + } + + return parseLongTimestamp(value, timeZoneKey.getZoneId()); + } + + private static long parseShortTimestamp(String value, ZoneId zoneId) + { + Matcher matcher = DATETIME_PATTERN.matcher(value); + if (!matcher.matches() || matcher.group("timezone") != null) { + throw new IllegalArgumentException("Invalid timestamp: " + value); + } + + String year = matcher.group("year"); + String month = matcher.group("month"); + String day = matcher.group("day"); + String hour = matcher.group("hour"); + String minute = matcher.group("minute"); + String second = matcher.group("second"); + String fraction = matcher.group("fraction"); + + long epochSecond = toEpochSecond(year, month, day, hour, minute, second, zoneId); + + int precision = 0; + long fractionValue = 0; + if (fraction != null) { + precision = fraction.length(); + fractionValue = Long.parseLong(fraction); + } + + if (precision <= 3) { + // scale to millis + return epochSecond * MILLISECONDS_PER_SECOND + rescale(fractionValue, precision, 3); + } + else if (precision <= MAX_SHORT_PRECISION) { + // scale to micros + return epochSecond * MICROSECONDS_PER_SECOND + rescale(fractionValue, precision, 6); + } + + throw new IllegalArgumentException(format("Cannot parse '%s' as short timestamp. Max allowed precision = %s", value, MAX_SHORT_PRECISION)); + } + + private static LongTimestamp parseLongTimestamp(String value, ZoneId zoneId) + { + Matcher matcher = DATETIME_PATTERN.matcher(value); + if (!matcher.matches() || matcher.group("timezone") != null) { + throw new IllegalArgumentException("Invalid timestamp: " + value); + } + + String year = matcher.group("year"); + String month = matcher.group("month"); + String day = matcher.group("day"); + String hour = matcher.group("hour"); + String minute = matcher.group("minute"); + String second = matcher.group("second"); + String fraction = matcher.group("fraction"); + + if (fraction == null || fraction.length() <= MAX_SHORT_PRECISION) { + throw new IllegalArgumentException(format("Cannot parse '%s' as long timestamp. Precision must be in the range [%s, %s]", value, MAX_SHORT_PRECISION + 1, MAX_PRECISION)); + } + + int precision = fraction.length(); + long epochSecond = toEpochSecond(year, month, day, hour, minute, second, zoneId); + long picoFraction = rescale(Long.parseLong(fraction), precision, 12); + + return longTimestamp(epochSecond, picoFraction); + } + + private static long toEpochSecond(String year, String month, String day, String hour, String minute, String second, ZoneId zoneId) + { + LocalDateTime timestamp = LocalDateTime.of( + Integer.parseInt(year), + Integer.parseInt(month), + Integer.parseInt(day), + hour == null ? 0 : Integer.parseInt(hour), + minute == null ? 0 : Integer.parseInt(minute), + second == null ? 0 : Integer.parseInt(second), + 0); + + // Only relevant for legacy timestamps. New timestamps are parsed using UTC, which doesn't + // have daylight savings transitions. TODO: remove once legacy timestamps are gone + List offsets = zoneId.getRules().getValidOffsets(timestamp); + if (offsets.isEmpty()) { + throw new IllegalArgumentException("Invalid timestamp due to daylight savings transition"); + } + + return timestamp.toEpochSecond(offsets.get(0)); + } + + public static LongTimestamp longTimestamp(long precision, Instant start) + { + checkArgument(precision > MAX_SHORT_PRECISION && precision <= MAX_PRECISION, "Precision is out of range"); + return new LongTimestamp( + start.getEpochSecond() * MICROSECONDS_PER_SECOND + start.getLong(MICRO_OF_SECOND), + (int) round((start.getNano() % PICOSECONDS_PER_NANOSECOND) * PICOSECONDS_PER_NANOSECOND, (int) (MAX_PRECISION - precision))); + } + + public static LongTimestamp longTimestamp(long epochSecond, long fractionInPicos) + { + return new LongTimestamp( + multiplyExact(epochSecond, MICROSECONDS_PER_SECOND) + fractionInPicos / PICOSECONDS_PER_MICROSECOND, + (int) (fractionInPicos % PICOSECONDS_PER_MICROSECOND)); + } +} diff --git a/presto-main/src/main/java/io/prestosql/type/TypeCoercion.java b/presto-main/src/main/java/io/prestosql/type/TypeCoercion.java index 2b6894cfb206d..18a5c16f88c53 100644 --- a/presto-main/src/main/java/io/prestosql/type/TypeCoercion.java +++ b/presto-main/src/main/java/io/prestosql/type/TypeCoercion.java @@ -20,6 +20,7 @@ import io.prestosql.spi.type.MapType; import io.prestosql.spi.type.RowType; import io.prestosql.spi.type.StandardTypes; +import io.prestosql.spi.type.TimestampType; import io.prestosql.spi.type.Type; import io.prestosql.spi.type.TypeSignature; import io.prestosql.spi.type.TypeSignatureParameter; @@ -42,7 +43,7 @@ import static io.prestosql.spi.type.RowType.Field; import static io.prestosql.spi.type.SmallintType.SMALLINT; import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TimestampType.createTimestampType; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; import static io.prestosql.spi.type.VarcharType.createVarcharType; @@ -155,6 +156,10 @@ private TypeCompatibility compatibility(Type fromType, Type toType) if (fromTypeBaseName.equals(StandardTypes.ROW)) { return typeCompatibilityForRow((RowType) fromType, (RowType) toType); } + if (fromTypeBaseName.equals(StandardTypes.TIMESTAMP)) { + Type commonSuperType = createTimestampType(Math.max(((TimestampType) fromType).getPrecision(), ((TimestampType) toType).getPrecision())); + return TypeCompatibility.compatible(commonSuperType, commonSuperType.equals(toType)); + } if (isCovariantParametrizedType(fromType)) { return typeCompatibilityForCovariantParametrizedType(fromType, toType); @@ -385,7 +390,7 @@ public Optional coerceTypeBase(Type sourceType, String resultTypeBase) case StandardTypes.DATE: { switch (resultTypeBase) { case StandardTypes.TIMESTAMP: - return Optional.of(TIMESTAMP); + return Optional.of(createTimestampType(0)); case StandardTypes.TIMESTAMP_WITH_TIME_ZONE: return Optional.of(TIMESTAMP_WITH_TIME_ZONE); default: diff --git a/presto-main/src/main/java/io/prestosql/util/DateTimeUtils.java b/presto-main/src/main/java/io/prestosql/util/DateTimeUtils.java index a713183210e09..56bc06b1b2ec3 100644 --- a/presto-main/src/main/java/io/prestosql/util/DateTimeUtils.java +++ b/presto-main/src/main/java/io/prestosql/util/DateTimeUtils.java @@ -21,7 +21,6 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.DurationFieldType; -import org.joda.time.LocalDateTime; import org.joda.time.MutablePeriod; import org.joda.time.Period; import org.joda.time.ReadWritablePeriod; @@ -36,9 +35,6 @@ import org.joda.time.format.PeriodFormatterBuilder; import org.joda.time.format.PeriodParser; -import java.lang.invoke.MethodHandle; -import java.lang.invoke.MethodHandles; -import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -73,8 +69,6 @@ public static String printDate(int days) return DATE_FORMATTER.print(TimeUnit.DAYS.toMillis(days)); } - private static final DateTimeFormatter LEGACY_TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER; - private static final DateTimeFormatter TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER; private static final DateTimeFormatter TIMESTAMP_WITH_TIME_ZONE_FORMATTER; private static final DateTimeFormatter TIMESTAMP_WITH_OR_WITHOUT_TIME_ZONE_FORMATTER; @@ -84,16 +78,6 @@ public static String printDate(int days) DateTimeFormat.forPattern("yyyy-M-d H:m").getParser(), DateTimeFormat.forPattern("yyyy-M-d H:m:s").getParser(), DateTimeFormat.forPattern("yyyy-M-d H:m:s.SSS").getParser()}; - DateTimePrinter timestampWithoutTimeZonePrinter = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS").getPrinter(); - LEGACY_TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER = new DateTimeFormatterBuilder() - .append(timestampWithoutTimeZonePrinter, timestampWithoutTimeZoneParser) - .toFormatter() - .withOffsetParsed(); - - TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER = new DateTimeFormatterBuilder() - .append(timestampWithoutTimeZonePrinter, timestampWithoutTimeZoneParser) - .toFormatter() - .withZoneUTC(); DateTimeParser[] timestampWithTimeZoneParser = { DateTimeFormat.forPattern("yyyy-M-dZ").getParser(), @@ -112,6 +96,7 @@ public static String printDate(int days) DateTimeFormat.forPattern("yyyy-M-d H:m:s ZZZ").getParser(), DateTimeFormat.forPattern("yyyy-M-d H:m:s.SSSZZZ").getParser(), DateTimeFormat.forPattern("yyyy-M-d H:m:s.SSS ZZZ").getParser()}; + DateTimePrinter timestampWithTimeZonePrinter = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS ZZZ").getPrinter(); TIMESTAMP_WITH_TIME_ZONE_FORMATTER = new DateTimeFormatterBuilder() .append(timestampWithTimeZonePrinter, timestampWithTimeZoneParser) @@ -126,78 +111,6 @@ public static String printDate(int days) .withOffsetParsed(); } - /** - * {@link LocalDateTime#getLocalMillis()} - */ - private static final MethodHandle getLocalMillis; - - static { - try { - Method getLocalMillisMethod = LocalDateTime.class.getDeclaredMethod("getLocalMillis"); - getLocalMillisMethod.setAccessible(true); - getLocalMillis = MethodHandles.lookup().unreflect(getLocalMillisMethod); - } - catch (ReflectiveOperationException e) { - throw new RuntimeException(e); - } - } - - /** - * Parse a legacy TIMESTAMP type. The string is interpreted in {@code timeZoneKey} zone. - */ - @Deprecated - public static long parseLegacyTimestamp(TimeZoneKey timeZoneKey, String value) - { - return LEGACY_TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER.withChronology(getChronology(timeZoneKey)).parseMillis(value); - } - - /** - * Parse a string (optionally containing a zone) as a value of TIMESTAMP type using legacy semantics. - * If the string doesn't specify a zone, it is interpreted in {@code timeZoneKey} zone. - * If the timestamp is in a gap in the provided timezone, the method throws an exception. - */ - @Deprecated - public static long convertToLegacyTimestamp(TimeZoneKey timeZoneKey, String value) - { - return TIMESTAMP_WITH_OR_WITHOUT_TIME_ZONE_FORMATTER.withChronology(getChronology(timeZoneKey)).parseMillis(value); - } - - /** - * Parse a string as a value of TIMESTAMP type. - * - * @return stack representation of TIMESTAMP type - */ - public static long parseTimestamp(String value) - { - LocalDateTime localDateTime = TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER.parseLocalDateTime(value); - try { - return (long) getLocalMillis.invokeExact(localDateTime); - } - catch (Throwable e) { - throw new RuntimeException(e); - } - } - - /** - * Parse a string (optionally containing a zone) as a value of TIMESTAMP type. - * If the string specifies a zone, the zone is discarded. - *

- * For example: {@code "2000-01-01 01:23:00"} is parsed to TIMESTAMP {@code 2000-01-01T01:23:00} - * and {@code "2000-01-01 01:23:00 +01:23"} is also parsed to TIMESTAMP {@code 2000-01-01T01:23:00.000}. - * - * @return stack representation of TIMESTAMP type - */ - public static long convertToTimestamp(String value) - { - LocalDateTime localDateTime = TIMESTAMP_WITH_OR_WITHOUT_TIME_ZONE_FORMATTER.parseLocalDateTime(value); - try { - return (long) getLocalMillis.invokeExact(localDateTime); - } - catch (Throwable e) { - throw new RuntimeException(e); - } - } - public static long parseTimestampWithTimeZone(String value) { DateTime dateTime = TIMESTAMP_WITH_TIME_ZONE_FORMATTER.withOffsetParsed().parseDateTime(value); @@ -227,38 +140,6 @@ public static String printTimestampWithTimeZone(long timestampWithTimeZone) return TIMESTAMP_WITH_TIME_ZONE_FORMATTER.withChronology(chronology).print(millis); } - public static String printTimestampWithoutTimeZone(long timestamp) - { - return TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER.print(timestamp); - } - - /** - * @deprecated applicable in legacy timestamp semantics only - */ - @Deprecated - public static String printTimestampWithoutTimeZone(TimeZoneKey timeZoneKey, long timestamp) - { - return LEGACY_TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER.withChronology(getChronology(timeZoneKey)).print(timestamp); - } - - public static boolean timestampHasTimeZone(String value) - { - try { - try { - TIMESTAMP_WITH_TIME_ZONE_FORMATTER.parseMillis(value); - return true; - } - catch (RuntimeException e) { - // `.withZoneUTC()` makes `timestampHasTimeZone` return value independent of JVM zone - TIMESTAMP_WITHOUT_TIME_ZONE_FORMATTER.withZoneUTC().parseMillis(value); - return false; - } - } - catch (RuntimeException e) { - throw new IllegalArgumentException(format("Invalid timestamp '%s'", value)); - } - } - private static final DateTimeFormatter TIME_FORMATTER; private static final DateTimeFormatter TIME_WITH_TIME_ZONE_FORMATTER; diff --git a/presto-main/src/main/java/io/prestosql/util/JsonUtil.java b/presto-main/src/main/java/io/prestosql/util/JsonUtil.java index e359ca033edd4..9aad57ccf4ef2 100644 --- a/presto-main/src/main/java/io/prestosql/util/JsonUtil.java +++ b/presto-main/src/main/java/io/prestosql/util/JsonUtil.java @@ -36,10 +36,12 @@ import io.prestosql.spi.type.Decimals; import io.prestosql.spi.type.DoubleType; import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.LongTimestamp; import io.prestosql.spi.type.MapType; import io.prestosql.spi.type.RealType; import io.prestosql.spi.type.RowType; import io.prestosql.spi.type.RowType.Field; +import io.prestosql.spi.type.ShortTimestampType; import io.prestosql.spi.type.SmallintType; import io.prestosql.spi.type.StandardTypes; import io.prestosql.spi.type.TimestampType; @@ -57,6 +59,8 @@ import java.io.InputStreamReader; import java.io.OutputStream; import java.math.BigDecimal; +import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -87,13 +91,13 @@ import static io.prestosql.spi.type.IntegerType.INTEGER; import static io.prestosql.spi.type.RealType.REAL; import static io.prestosql.spi.type.SmallintType.SMALLINT; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static io.prestosql.spi.type.TinyintType.TINYINT; import static io.prestosql.type.JsonType.JSON; +import static io.prestosql.type.Timestamps.formatTimestamp; +import static io.prestosql.type.Timestamps.scaleEpochMillisToMicros; import static io.prestosql.type.TypeUtils.hashPosition; import static io.prestosql.type.TypeUtils.positionEqualsPosition; import static io.prestosql.util.DateTimeUtils.printDate; -import static io.prestosql.util.DateTimeUtils.printTimestampWithoutTimeZone; import static io.prestosql.util.JsonUtil.ObjectKeyProvider.createObjectKeyProvider; import static it.unimi.dsi.fastutil.HashCommon.arraySize; import static java.lang.Float.floatToRawIntBits; @@ -288,7 +292,7 @@ static JsonGeneratorWriter createJsonGeneratorWriter(Type type) return new JsonJsonGeneratorWriter(); } if (type instanceof TimestampType) { - return new TimestampJsonGeneratorWriter(); + return new TimestampJsonGeneratorWriter((TimestampType) type); } if (type instanceof DateType) { return new DateGeneratorWriter(); @@ -499,6 +503,13 @@ public void writeJsonValue(JsonGenerator jsonGenerator, Block block, int positio private static class TimestampJsonGeneratorWriter implements JsonGeneratorWriter { + private final TimestampType type; + + public TimestampJsonGeneratorWriter(TimestampType type) + { + this.type = type; + } + @Override public void writeJsonValue(JsonGenerator jsonGenerator, Block block, int position, ConnectorSession session) throws IOException @@ -507,14 +518,28 @@ public void writeJsonValue(JsonGenerator jsonGenerator, Block block, int positio jsonGenerator.writeNull(); } else { - long value = TIMESTAMP.getLong(block, position); + long epochMicros; + int fraction; - if (session.isLegacyTimestamp()) { - jsonGenerator.writeString(printTimestampWithoutTimeZone(session.getTimeZoneKey(), value)); + if (type instanceof ShortTimestampType) { + epochMicros = type.getLong(block, position); + if (type.getPrecision() <= 3) { + epochMicros = scaleEpochMillisToMicros(epochMicros); + } + fraction = 0; } else { - jsonGenerator.writeString(printTimestampWithoutTimeZone(value)); + LongTimestamp timestamp = (LongTimestamp) type.getObject(block, position); + epochMicros = timestamp.getEpochMicros(); + fraction = timestamp.getPicosOfMicro(); } + + ZoneId zoneId = ZoneOffset.UTC; + if (session.isLegacyTimestamp()) { + zoneId = session.getTimeZoneKey().getZoneId(); + } + + jsonGenerator.writeString(formatTimestamp(type.getPrecision(), epochMicros, fraction, zoneId)); } } } diff --git a/presto-main/src/test/java/io/prestosql/block/AbstractTestBlock.java b/presto-main/src/test/java/io/prestosql/block/AbstractTestBlock.java index b96e00ecab947..b29581cd32f52 100644 --- a/presto-main/src/test/java/io/prestosql/block/AbstractTestBlock.java +++ b/presto-main/src/test/java/io/prestosql/block/AbstractTestBlock.java @@ -402,7 +402,7 @@ protected boolean isSliceAccessSupported() return true; } - // Subclasses should implement this method to customize how the position is compared + // Subclasses can implement this method to customize how the position is compared // with the expected bytes protected void assertPositionEquals(Block block, int position, Slice expectedBytes) { diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctions.java b/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctions.java index e41f73b4284af..6018da966fa74 100644 --- a/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctions.java +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctions.java @@ -22,7 +22,6 @@ import java.time.Instant; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; -import static io.prestosql.spi.type.VarcharType.createVarcharType; public class TestDateTimeFunctions extends TestDateTimeFunctionsBase diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctionsBase.java b/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctionsBase.java index e929ce04d2f36..cadbfe17b0e4b 100644 --- a/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctionsBase.java +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/TestDateTimeFunctionsBase.java @@ -58,6 +58,7 @@ import static io.prestosql.spi.type.TimeZoneKey.UTC_KEY; import static io.prestosql.spi.type.TimeZoneKey.getTimeZoneKey; import static io.prestosql.spi.type.TimeZoneKey.getTimeZoneKeyForOffset; +import static io.prestosql.spi.type.TimestampType.createTimestampType; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.VarcharType.VARCHAR; import static io.prestosql.spi.type.VarcharType.createVarcharType; @@ -133,7 +134,7 @@ protected TestDateTimeFunctionsBase(boolean legacyTimestamp) @Test public void testToIso8601ForTimestampWithoutTimeZone() { - assertFunction("to_iso8601(" + TIMESTAMP_LITERAL + ")", createVarcharType(29), TIMESTAMP_ISO8601_STRING_NO_TIME_ZONE); + assertFunction("to_iso8601(" + TIMESTAMP_LITERAL + ")", createVarcharType(26), TIMESTAMP_ISO8601_STRING_NO_TIME_ZONE); } @Test @@ -883,67 +884,67 @@ public void testDateParse() { assertFunction("date_parse('2013', '%Y')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 1, 1, 0, 0, 0, 0, session)); + sqlTimestampOf(3, 2013, 1, 1, 0, 0, 0, 0, session)); assertFunction("date_parse('2013-05', '%Y-%m')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 1, 0, 0, 0, 0, session)); + sqlTimestampOf(3, 2013, 5, 1, 0, 0, 0, 0, session)); assertFunction("date_parse('2013-05-17', '%Y-%m-%d')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 0, 0, 0, 0, session)); + sqlTimestampOf(3, 2013, 5, 17, 0, 0, 0, 0, session)); assertFunction("date_parse('2013-05-17 12:35:10', '%Y-%m-%d %h:%i:%s')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 0, 35, 10, 0, session)); + sqlTimestampOf(3, 2013, 5, 17, 0, 35, 10, 0, session)); assertFunction("date_parse('2013-05-17 12:35:10 PM', '%Y-%m-%d %h:%i:%s %p')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 12, 35, 10, 0, session)); + sqlTimestampOf(3, 2013, 5, 17, 12, 35, 10, 0, session)); assertFunction("date_parse('2013-05-17 12:35:10 AM', '%Y-%m-%d %h:%i:%s %p')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 0, 35, 10, 0, session)); + sqlTimestampOf(3, 2013, 5, 17, 0, 35, 10, 0, session)); assertFunction("date_parse('2013-05-17 00:35:10', '%Y-%m-%d %H:%i:%s')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 0, 35, 10, 0, session)); + sqlTimestampOf(3, 2013, 5, 17, 0, 35, 10, 0, session)); assertFunction("date_parse('2013-05-17 23:35:10', '%Y-%m-%d %H:%i:%s')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 23, 35, 10, 0, session)); + sqlTimestampOf(3, 2013, 5, 17, 23, 35, 10, 0, session)); assertFunction("date_parse('abc 2013-05-17 fff 23:35:10 xyz', 'abc %Y-%m-%d fff %H:%i:%s xyz')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 23, 35, 10, 0, session)); + sqlTimestampOf(3, 2013, 5, 17, 23, 35, 10, 0, session)); assertFunction("date_parse('2013 14', '%Y %y')", TimestampType.TIMESTAMP, - sqlTimestampOf(2014, 1, 1, 0, 0, 0, 0, session)); + sqlTimestampOf(3, 2014, 1, 1, 0, 0, 0, 0, session)); assertFunction("date_parse('1998 53', '%x %v')", TimestampType.TIMESTAMP, - sqlTimestampOf(1998, 12, 28, 0, 0, 0, 0, session)); + sqlTimestampOf(3, 1998, 12, 28, 0, 0, 0, 0, session)); assertFunction("date_parse('1.1', '%s.%f')", TimestampType.TIMESTAMP, - sqlTimestampOf(1970, 1, 1, 0, 0, 1, 100, session)); + sqlTimestampOf(3, 1970, 1, 1, 0, 0, 1, 100, session)); assertFunction("date_parse('1.01', '%s.%f')", TimestampType.TIMESTAMP, - sqlTimestampOf(1970, 1, 1, 0, 0, 1, 10, session)); + sqlTimestampOf(3, 1970, 1, 1, 0, 0, 1, 10, session)); assertFunction("date_parse('1.2006', '%s.%f')", TimestampType.TIMESTAMP, - sqlTimestampOf(1970, 1, 1, 0, 0, 1, 200, session)); + sqlTimestampOf(3, 1970, 1, 1, 0, 0, 1, 200, session)); assertFunction("date_parse('59.123456789', '%s.%f')", TimestampType.TIMESTAMP, - sqlTimestampOf(1970, 1, 1, 0, 0, 59, 123, session)); + sqlTimestampOf(3, 1970, 1, 1, 0, 0, 59, 123, session)); assertFunction("date_parse('0', '%k')", TimestampType.TIMESTAMP, - sqlTimestampOf(1970, 1, 1, 0, 0, 0, 0, session)); + sqlTimestampOf(3, 1970, 1, 1, 0, 0, 0, 0, session)); assertFunction("date_parse('28-JAN-16 11.45.46.421000 PM','%d-%b-%y %l.%i.%s.%f %p')", TimestampType.TIMESTAMP, - sqlTimestampOf(2016, 1, 28, 23, 45, 46, 421, session)); + sqlTimestampOf(3, 2016, 1, 28, 23, 45, 46, 421, session)); assertFunction("date_parse('11-DEC-70 11.12.13.456000 AM','%d-%b-%y %l.%i.%s.%f %p')", TimestampType.TIMESTAMP, - sqlTimestampOf(1970, 12, 11, 11, 12, 13, 456, session)); + sqlTimestampOf(3, 1970, 12, 11, 11, 12, 13, 456, session)); assertFunction("date_parse('31-MAY-69 04.59.59.999000 AM','%d-%b-%y %l.%i.%s.%f %p')", TimestampType.TIMESTAMP, - sqlTimestampOf(2069, 5, 31, 4, 59, 59, 999, session)); + sqlTimestampOf(3, 2069, 5, 31, 4, 59, 59, 999, session)); assertInvalidFunction("date_parse('', '%D')", "%D not supported in date format string"); assertInvalidFunction("date_parse('', '%U')", "%U not supported in date format string"); @@ -983,10 +984,10 @@ public void testLocale() localeAssertions.assertFunction("date_parse('2013-05-17 12:35:10 오후', '%Y-%m-%d %h:%i:%s %p')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 12, 35, 10, 0, localeSession)); + sqlTimestampOf(3, 2013, 5, 17, 12, 35, 10, 0, localeSession)); localeAssertions.assertFunction("date_parse('2013-05-17 12:35:10 오전', '%Y-%m-%d %h:%i:%s %p')", TimestampType.TIMESTAMP, - sqlTimestampOf(2013, 5, 17, 0, 35, 10, 0, localeSession)); + sqlTimestampOf(3, 2013, 5, 17, 0, 35, 10, 0, localeSession)); localeAssertions.assertFunction("parse_datetime('2013-05-17 12:35:10 오후', 'yyyy-MM-dd hh:mm:ss a')", TIMESTAMP_WITH_TIME_ZONE, @@ -1021,10 +1022,10 @@ public void testDateTimeOutputString() assertFunctionString("time '23:59:59.999 Asia/Kathmandu'", TIME_WITH_TIME_ZONE, "23:59:59.999 Asia/Kathmandu"); // SqlTimestamp - assertFunctionString("timestamp '0000-01-02 01:02:03'", TimestampType.TIMESTAMP, "0000-01-02 01:02:03.000"); - assertFunctionString("timestamp '2012-12-31 00:00:00'", TimestampType.TIMESTAMP, "2012-12-31 00:00:00.000"); - assertFunctionString("timestamp '1234-05-06 23:23:23.233'", TimestampType.TIMESTAMP, "1234-05-06 23:23:23.233"); - assertFunctionString("timestamp '2333-02-23 23:59:59.999'", TimestampType.TIMESTAMP, "2333-02-23 23:59:59.999"); + assertFunctionString("timestamp '0000-01-02 01:02:03'", createTimestampType(0), "0000-01-02 01:02:03"); + assertFunctionString("timestamp '2012-12-31 00:00:00'", createTimestampType(0), "2012-12-31 00:00:00"); + assertFunctionString("timestamp '1234-05-06 23:23:23.233'", createTimestampType(3), "1234-05-06 23:23:23.233"); + assertFunctionString("timestamp '2333-02-23 23:59:59.999'", createTimestampType(3), "2333-02-23 23:59:59.999"); // SqlTimestampWithTimeZone assertFunctionString("timestamp '2012-12-31 00:00:00 UTC'", TIMESTAMP_WITH_TIME_ZONE, "2012-12-31 00:00:00.000 UTC"); diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestDateTrunc.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestDateTrunc.java new file mode 100644 index 0000000000000..5c31c0a58a7a7 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestDateTrunc.java @@ -0,0 +1,214 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.Session; +import io.prestosql.sql.query.QueryAssertions; +import io.prestosql.testing.TestingSession; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static org.assertj.core.api.Assertions.assertThat; + +public abstract class BaseTestDateTrunc +{ + private final boolean legacyTimestamp; + private QueryAssertions assertions; + + protected BaseTestDateTrunc(boolean legacyTimestamp) + { + this.legacyTimestamp = legacyTimestamp; + } + + @BeforeClass + public void init() + { + Session session = testSessionBuilder() + .setSystemProperty("legacy_timestamp", String.valueOf(legacyTimestamp)) + .setTimeZoneKey(TestingSession.DEFAULT_TIME_ZONE_KEY) + .build(); + assertions = new QueryAssertions(session); + } + + @AfterClass(alwaysRun = true) + public void teardown() + { + assertions.close(); + assertions = null; + } + + @Test + public void testDateTruncYear() + { + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-01-01 00:00:00'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-01-01 00:00:00.0'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-01-01 00:00:00.00'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-01-01 00:00:00.000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-01-01 00:00:00.0000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-01-01 00:00:00.00000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-01-01 00:00:00.000000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-01-01 00:00:00.0000000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-01-01 00:00:00.00000000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-01-01 00:00:00.000000000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-01-01 00:00:00.0000000000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-01-01 00:00:00.00000000000'"); + assertThat(assertions.expression("date_trunc('year', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-01-01 00:00:00.000000000000'"); + } + + @Test + public void testDateTruncQuarter() + { + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-04-01 00:00:00'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-04-01 00:00:00.0'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-04-01 00:00:00.00'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-04-01 00:00:00.000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-04-01 00:00:00.0000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-04-01 00:00:00.00000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-04-01 00:00:00.000000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-04-01 00:00:00.0000000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-04-01 00:00:00.00000000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-04-01 00:00:00.000000000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-04-01 00:00:00.0000000000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-04-01 00:00:00.00000000000'"); + assertThat(assertions.expression("date_trunc('quarter', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-04-01 00:00:00.000000000000'"); + } + + @Test + public void testDateTruncMonth() + { + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-01 00:00:00'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-01 00:00:00.0'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-05-01 00:00:00.00'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-05-01 00:00:00.000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-05-01 00:00:00.0000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-05-01 00:00:00.00000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-05-01 00:00:00.000000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-05-01 00:00:00.0000000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-05-01 00:00:00.00000000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-05-01 00:00:00.000000000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-05-01 00:00:00.0000000000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-05-01 00:00:00.00000000000'"); + assertThat(assertions.expression("date_trunc('month', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-05-01 00:00:00.000000000000'"); + } + + @Test + public void testDateTruncWeek() + { + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-04 00:00:00'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-04 00:00:00.0'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-05-04 00:00:00.00'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-05-04 00:00:00.000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-05-04 00:00:00.0000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-05-04 00:00:00.00000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-05-04 00:00:00.000000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-05-04 00:00:00.0000000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-05-04 00:00:00.00000000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-05-04 00:00:00.000000000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-05-04 00:00:00.0000000000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-05-04 00:00:00.00000000000'"); + assertThat(assertions.expression("date_trunc('week', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-05-04 00:00:00.000000000000'"); + } + + @Test + public void testDateTruncDay() + { + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-10 00:00:00'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-10 00:00:00.0'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-05-10 00:00:00.00'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-05-10 00:00:00.000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-05-10 00:00:00.0000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-05-10 00:00:00.00000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-05-10 00:00:00.000000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-05-10 00:00:00.0000000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-05-10 00:00:00.00000000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-05-10 00:00:00.000000000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-05-10 00:00:00.0000000000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-05-10 00:00:00.00000000000'"); + assertThat(assertions.expression("date_trunc('day', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-05-10 00:00:00.000000000000'"); + } + + @Test + public void testDateTruncHour() + { + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-10 12:00:00'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-10 12:00:00.0'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-05-10 12:00:00.00'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-05-10 12:00:00.000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-05-10 12:00:00.0000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-05-10 12:00:00.00000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-05-10 12:00:00.000000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-05-10 12:00:00.0000000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-05-10 12:00:00.00000000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-05-10 12:00:00.000000000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-05-10 12:00:00.0000000000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-05-10 12:00:00.00000000000'"); + assertThat(assertions.expression("date_trunc('hour', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-05-10 12:00:00.000000000000'"); + } + + @Test + public void testDateTruncMinute() + { + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-10 12:34:00'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-10 12:34:00.0'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-05-10 12:34:00.00'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-05-10 12:34:00.000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-05-10 12:34:00.0000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-05-10 12:34:00.00000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-05-10 12:34:00.000000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-05-10 12:34:00.0000000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-05-10 12:34:00.00000000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-05-10 12:34:00.000000000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-05-10 12:34:00.0000000000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-05-10 12:34:00.00000000000'"); + assertThat(assertions.expression("date_trunc('minute', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-05-10 12:34:00.000000000000'"); + } + + @Test + public void testDateTruncSecond() + { + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-10 12:34:56.0'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-05-10 12:34:56.00'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-05-10 12:34:56.000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-05-10 12:34:56.0000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-05-10 12:34:56.00000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-05-10 12:34:56.000000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-05-10 12:34:56.0000000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-05-10 12:34:56.00000000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-05-10 12:34:56.000000000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-05-10 12:34:56.0000000000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-05-10 12:34:56.00000000000'"); + assertThat(assertions.expression("date_trunc('second', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-05-10 12:34:56.000000000000'"); + } + + @Test + public void testDateTruncMilliSecond() + { + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.12')")).matches("TIMESTAMP '2020-05-10 12:34:56.12'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.123')")).matches("TIMESTAMP '2020-05-10 12:34:56.123'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("TIMESTAMP '2020-05-10 12:34:56.1230'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("TIMESTAMP '2020-05-10 12:34:56.12300'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("TIMESTAMP '2020-05-10 12:34:56.123000'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("TIMESTAMP '2020-05-10 12:34:56.1230000'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("TIMESTAMP '2020-05-10 12:34:56.12300000'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("TIMESTAMP '2020-05-10 12:34:56.123000000'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("TIMESTAMP '2020-05-10 12:34:56.1230000000'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("TIMESTAMP '2020-05-10 12:34:56.12300000000'"); + assertThat(assertions.expression("date_trunc('millisecond', TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("TIMESTAMP '2020-05-10 12:34:56.123000000000'"); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestExtract.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestExtract.java new file mode 100644 index 0000000000000..802ffddcdefa8 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestExtract.java @@ -0,0 +1,283 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.Session; +import io.prestosql.sql.query.QueryAssertions; +import io.prestosql.testing.TestingSession; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static org.assertj.core.api.Assertions.assertThat; + +public abstract class BaseTestExtract +{ + private final boolean legacyTimestamp; + private QueryAssertions assertions; + + protected BaseTestExtract(boolean legacyTimestamp) + { + this.legacyTimestamp = legacyTimestamp; + } + + @BeforeClass + public void init() + { + Session session = testSessionBuilder() + .setSystemProperty("legacy_timestamp", String.valueOf(legacyTimestamp)) + .setTimeZoneKey(TestingSession.DEFAULT_TIME_ZONE_KEY) + .build(); + assertions = new QueryAssertions(session); + } + + @AfterClass(alwaysRun = true) + public void teardown() + { + assertions.close(); + assertions = null; + } + + @Test + public void testYear() + { + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '2020'"); + } + + @Test + public void testMonth() + { + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '5'"); + assertThat(assertions.expression("month(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '5'"); + } + + @Test + public void testDay() + { + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '10'"); + assertThat(assertions.expression("day(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '10'"); + } + + @Test + public void testHour() + { + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '12'"); + assertThat(assertions.expression("hour(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '12'"); + } + + @Test + public void testMinute() + { + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '34'"); + assertThat(assertions.expression("minute(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '34'"); + } + + @Test + public void testSecond() + { + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '56'"); + + // negative epoch + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.1')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.12')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.123')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.1234')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.12345')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.123456')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.1234567')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.12345678')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.123456789')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.1234567890')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.12345678901')")).matches("BIGINT '56'"); + assertThat(assertions.expression("second(TIMESTAMP '1500-05-10 12:34:56.123456789012')")).matches("BIGINT '56'"); + } + + @Test + public void testMillisecond() + { + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '0'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '100'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '120'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '123'"); + assertThat(assertions.expression("millisecond(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '123'"); + } + + @Test + public void testDayOfWeek() + { + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '7'"); + assertThat(assertions.expression("day_of_week(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '7'"); + } + + @Test + public void testDayOfYear() + { + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '131'"); + assertThat(assertions.expression("day_of_year(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '131'"); + } + + @Test + public void testQuarter() + { + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '2'"); + assertThat(assertions.expression("quarter(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '2'"); + } + + @Test + public void testWeekOfYear() + { + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '19'"); + assertThat(assertions.expression("week_of_year(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '19'"); + } + + @Test + public void testYearOfWeek() + { + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("BIGINT '2020'"); + assertThat(assertions.expression("year_of_week(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("BIGINT '2020'"); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestOperators.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestOperators.java new file mode 100644 index 0000000000000..ee81e44860f2e --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestOperators.java @@ -0,0 +1,510 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.Session; +import io.prestosql.sql.query.QueryAssertions; +import io.prestosql.testing.TestingSession; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static org.assertj.core.api.Assertions.assertThat; + +public abstract class BaseTestOperators +{ + private final boolean legacyTimestamp; + private QueryAssertions assertions; + + protected BaseTestOperators(boolean legacyTimestamp) + { + this.legacyTimestamp = legacyTimestamp; + } + + @BeforeClass + public void init() + { + Session session = testSessionBuilder() + .setSystemProperty("legacy_timestamp", String.valueOf(legacyTimestamp)) + .setTimeZoneKey(TestingSession.DEFAULT_TIME_ZONE_KEY) + .build(); + assertions = new QueryAssertions(session); + } + + @AfterClass(alwaysRun = true) + public void teardown() + { + assertions.close(); + assertions = null; + } + + @Test + public void testEqual() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' = TIMESTAMP '2020-05-01 12:34:56'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' = TIMESTAMP '2020-05-01 12:34:56.1'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' = TIMESTAMP '2020-05-01 12:34:56.12'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' = TIMESTAMP '2020-05-01 12:34:56.123'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' = TIMESTAMP '2020-05-01 12:34:56.1234'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' = TIMESTAMP '2020-05-01 12:34:56.12345'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' = TIMESTAMP '2020-05-01 12:34:56.123456'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' = TIMESTAMP '2020-05-01 12:34:56.1234567'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' = TIMESTAMP '2020-05-01 12:34:56.12345678'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' = TIMESTAMP '2020-05-01 12:34:56.123456789'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' = TIMESTAMP '2020-05-01 12:34:56.1234567890'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' = TIMESTAMP '2020-05-01 12:34:56.12345678901'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' = TIMESTAMP '2020-05-01 12:34:56.123456789012'")).isEqualTo(true); + } + + @Test + public void testNotEqual() + { + // false cases + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' <> TIMESTAMP '2020-05-01 12:34:56'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' <> TIMESTAMP '2020-05-01 12:34:56.1'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' <> TIMESTAMP '2020-05-01 12:34:56.12'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' <> TIMESTAMP '2020-05-01 12:34:56.123'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' <> TIMESTAMP '2020-05-01 12:34:56.1234'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' <> TIMESTAMP '2020-05-01 12:34:56.12345'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' <> TIMESTAMP '2020-05-01 12:34:56.123456'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' <> TIMESTAMP '2020-05-01 12:34:56.1234567'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' <> TIMESTAMP '2020-05-01 12:34:56.12345678'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' <> TIMESTAMP '2020-05-01 12:34:56.123456789'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' <> TIMESTAMP '2020-05-01 12:34:56.1234567890'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' <> TIMESTAMP '2020-05-01 12:34:56.12345678901'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' <> TIMESTAMP '2020-05-01 12:34:56.123456789012'")).isEqualTo(false); + + // difference in the high-order data + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' <> TIMESTAMP '2021-05-01 12:34:56'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' <> TIMESTAMP '2021-05-01 12:34:56.1'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' <> TIMESTAMP '2021-05-01 12:34:56.12'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' <> TIMESTAMP '2021-05-01 12:34:56.123'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' <> TIMESTAMP '2021-05-01 12:34:56.1234'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' <> TIMESTAMP '2021-05-01 12:34:56.12345'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' <> TIMESTAMP '2021-05-01 12:34:56.123456'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' <> TIMESTAMP '2021-05-01 12:34:56.1234567'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' <> TIMESTAMP '2021-05-01 12:34:56.12345678'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' <> TIMESTAMP '2021-05-01 12:34:56.123456789'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' <> TIMESTAMP '2021-05-01 12:34:56.1234567890'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' <> TIMESTAMP '2021-05-01 12:34:56.12345678901'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' <> TIMESTAMP '2021-05-01 12:34:56.123456789012'")).isEqualTo(true); + + // difference in the low-order data + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' <> TIMESTAMP '2020-05-01 12:34:57'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' <> TIMESTAMP '2020-05-01 12:34:56.2'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' <> TIMESTAMP '2020-05-01 12:34:56.13'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' <> TIMESTAMP '2020-05-01 12:34:56.124'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' <> TIMESTAMP '2020-05-01 12:34:56.1235'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' <> TIMESTAMP '2020-05-01 12:34:56.12346'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' <> TIMESTAMP '2020-05-01 12:34:56.123457'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' <> TIMESTAMP '2020-05-01 12:34:56.1234568'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' <> TIMESTAMP '2020-05-01 12:34:56.12345679'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' <> TIMESTAMP '2020-05-01 12:34:56.123456781'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' <> TIMESTAMP '2020-05-01 12:34:56.1234567891'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' <> TIMESTAMP '2020-05-01 12:34:56.12345678902'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' <> TIMESTAMP '2020-05-01 12:34:56.123456789013'")).isEqualTo(true); + } + + @Test + public void testDistinctFrom() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.1'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.12'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.123'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.1234'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.12345'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.123456'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.1234567'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.12345678'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.123456789'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.1234567890'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.12345678901'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.123456789012'")).isEqualTo(false); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:55'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.0'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.11'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.122'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.1233'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.12344'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.123455'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.1234566'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.12345677'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.123456788'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.1234567899'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.12345678900'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' IS DISTINCT FROM TIMESTAMP '2020-05-01 12:34:56.123456789011'")).isEqualTo(true); + } + + @Test + public void testLessThan() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' < TIMESTAMP '2020-05-01 12:34:57'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' < TIMESTAMP '2020-05-01 12:34:56.2'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' < TIMESTAMP '2020-05-01 12:34:56.13'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' < TIMESTAMP '2020-05-01 12:34:56.124'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' < TIMESTAMP '2020-05-01 12:34:56.1235'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' < TIMESTAMP '2020-05-01 12:34:56.12346'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' < TIMESTAMP '2020-05-01 12:34:56.123457'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' < TIMESTAMP '2020-05-01 12:34:56.1234568'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' < TIMESTAMP '2020-05-01 12:34:56.12345679'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' < TIMESTAMP '2020-05-01 12:34:56.123456790'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' < TIMESTAMP '2020-05-01 12:34:56.1234567891'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' < TIMESTAMP '2020-05-01 12:34:56.12345678902'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' < TIMESTAMP '2020-05-01 12:34:56.123456789013'")).isEqualTo(true); + + // false cases + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' < TIMESTAMP '2020-05-01 12:34:56'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' < TIMESTAMP '2020-05-01 12:34:56.1'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' < TIMESTAMP '2020-05-01 12:34:56.12'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' < TIMESTAMP '2020-05-01 12:34:56.123'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' < TIMESTAMP '2020-05-01 12:34:56.1234'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' < TIMESTAMP '2020-05-01 12:34:56.12345'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' < TIMESTAMP '2020-05-01 12:34:56.123456'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' < TIMESTAMP '2020-05-01 12:34:56.1234567'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' < TIMESTAMP '2020-05-01 12:34:56.12345678'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' < TIMESTAMP '2020-05-01 12:34:56.123456789'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' < TIMESTAMP '2020-05-01 12:34:56.1234567890'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' < TIMESTAMP '2020-05-01 12:34:56.12345678901'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' < TIMESTAMP '2020-05-01 12:34:56.123456789012'")).isEqualTo(false); + } + + @Test + public void testGreaterThan() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' > TIMESTAMP '2020-05-01 12:34:55'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' > TIMESTAMP '2020-05-01 12:34:55.9'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' > TIMESTAMP '2020-05-01 12:34:56.11'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' > TIMESTAMP '2020-05-01 12:34:56.122'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' > TIMESTAMP '2020-05-01 12:34:56.1233'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' > TIMESTAMP '2020-05-01 12:34:56.12344'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' > TIMESTAMP '2020-05-01 12:34:56.123455'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' > TIMESTAMP '2020-05-01 12:34:56.1234566'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' > TIMESTAMP '2020-05-01 12:34:56.12345677'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' > TIMESTAMP '2020-05-01 12:34:56.123456788'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' > TIMESTAMP '2020-05-01 12:34:56.1234567889'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' > TIMESTAMP '2020-05-01 12:34:56.12345678899'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' > TIMESTAMP '2020-05-01 12:34:56.123456789011'")).isEqualTo(true); + + // false cases + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' > TIMESTAMP '2020-05-01 12:34:56'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' > TIMESTAMP '2020-05-01 12:34:56.1'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' > TIMESTAMP '2020-05-01 12:34:56.12'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' > TIMESTAMP '2020-05-01 12:34:56.123'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' > TIMESTAMP '2020-05-01 12:34:56.1234'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' > TIMESTAMP '2020-05-01 12:34:56.12345'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' > TIMESTAMP '2020-05-01 12:34:56.123456'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' > TIMESTAMP '2020-05-01 12:34:56.1234567'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' > TIMESTAMP '2020-05-01 12:34:56.12345678'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' > TIMESTAMP '2020-05-01 12:34:56.123456789'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' > TIMESTAMP '2020-05-01 12:34:56.1234567890'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' > TIMESTAMP '2020-05-01 12:34:56.12345678901'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' > TIMESTAMP '2020-05-01 12:34:56.123456789012'")).isEqualTo(false); + } + + @Test + public void testLessThanOrEquals() + { + // equality + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' <= TIMESTAMP '2020-05-01 12:34:56'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' <= TIMESTAMP '2020-05-01 12:34:56.1'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' <= TIMESTAMP '2020-05-01 12:34:56.12'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' <= TIMESTAMP '2020-05-01 12:34:56.123'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' <= TIMESTAMP '2020-05-01 12:34:56.1234'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' <= TIMESTAMP '2020-05-01 12:34:56.12345'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' <= TIMESTAMP '2020-05-01 12:34:56.123456'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' <= TIMESTAMP '2020-05-01 12:34:56.1234567'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' <= TIMESTAMP '2020-05-01 12:34:56.12345678'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' <= TIMESTAMP '2020-05-01 12:34:56.123456789'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' <= TIMESTAMP '2020-05-01 12:34:56.1234567890'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' <= TIMESTAMP '2020-05-01 12:34:56.12345678901'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' <= TIMESTAMP '2020-05-01 12:34:56.123456789012'")).isEqualTo(true); + + // less than + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' <= TIMESTAMP '2020-05-01 12:34:57'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' <= TIMESTAMP '2020-05-01 12:34:56.2'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' <= TIMESTAMP '2020-05-01 12:34:56.13'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' <= TIMESTAMP '2020-05-01 12:34:56.124'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' <= TIMESTAMP '2020-05-01 12:34:56.1235'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' <= TIMESTAMP '2020-05-01 12:34:56.12346'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' <= TIMESTAMP '2020-05-01 12:34:56.123457'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' <= TIMESTAMP '2020-05-01 12:34:56.1234568'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' <= TIMESTAMP '2020-05-01 12:34:56.12345679'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' <= TIMESTAMP '2020-05-01 12:34:56.123456790'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' <= TIMESTAMP '2020-05-01 12:34:56.1234567891'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' <= TIMESTAMP '2020-05-01 12:34:56.12345678902'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' <= TIMESTAMP '2020-05-01 12:34:56.123456789013'")).isEqualTo(true); + + // false cases + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' <= TIMESTAMP '2020-05-01 12:34:55'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' <= TIMESTAMP '2020-05-01 12:34:56.0'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' <= TIMESTAMP '2020-05-01 12:34:56.11'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' <= TIMESTAMP '2020-05-01 12:34:56.122'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' <= TIMESTAMP '2020-05-01 12:34:56.1233'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' <= TIMESTAMP '2020-05-01 12:34:56.12344'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' <= TIMESTAMP '2020-05-01 12:34:56.123454'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' <= TIMESTAMP '2020-05-01 12:34:56.1234566'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' <= TIMESTAMP '2020-05-01 12:34:56.12345677'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' <= TIMESTAMP '2020-05-01 12:34:56.123456788'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' <= TIMESTAMP '2020-05-01 12:34:56.1234567889'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' <= TIMESTAMP '2020-05-01 12:34:56.12345678900'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' <= TIMESTAMP '2020-05-01 12:34:56.123456789011'")).isEqualTo(false); + } + + @Test + public void testGreaterThanOrEquals() + { + // equality + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' >= TIMESTAMP '2020-05-01 12:34:56'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' >= TIMESTAMP '2020-05-01 12:34:56.1'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' >= TIMESTAMP '2020-05-01 12:34:56.12'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' >= TIMESTAMP '2020-05-01 12:34:56.123'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' >= TIMESTAMP '2020-05-01 12:34:56.1234'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' >= TIMESTAMP '2020-05-01 12:34:56.12345'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' >= TIMESTAMP '2020-05-01 12:34:56.123456'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' >= TIMESTAMP '2020-05-01 12:34:56.1234567'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' >= TIMESTAMP '2020-05-01 12:34:56.12345678'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' >= TIMESTAMP '2020-05-01 12:34:56.123456789'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' >= TIMESTAMP '2020-05-01 12:34:56.1234567890'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' >= TIMESTAMP '2020-05-01 12:34:56.12345678901'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' >= TIMESTAMP '2020-05-01 12:34:56.123456789012'")).isEqualTo(true); + + // greater than + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' >= TIMESTAMP '2020-05-01 12:34:55'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' >= TIMESTAMP '2020-05-01 12:34:55.9'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' >= TIMESTAMP '2020-05-01 12:34:56.11'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' >= TIMESTAMP '2020-05-01 12:34:56.122'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' >= TIMESTAMP '2020-05-01 12:34:56.1233'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' >= TIMESTAMP '2020-05-01 12:34:56.12344'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' >= TIMESTAMP '2020-05-01 12:34:56.123455'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' >= TIMESTAMP '2020-05-01 12:34:56.1234566'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' >= TIMESTAMP '2020-05-01 12:34:56.12345677'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' >= TIMESTAMP '2020-05-01 12:34:56.123456788'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' >= TIMESTAMP '2020-05-01 12:34:56.1234567889'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' >= TIMESTAMP '2020-05-01 12:34:56.12345678899'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' >= TIMESTAMP '2020-05-01 12:34:56.123456789011'")).isEqualTo(true); + + // false cases + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' >= TIMESTAMP '2020-05-01 12:34:57'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' >= TIMESTAMP '2020-05-01 12:34:56.2'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' >= TIMESTAMP '2020-05-01 12:34:56.13'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' >= TIMESTAMP '2020-05-01 12:34:56.124'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' >= TIMESTAMP '2020-05-01 12:34:56.1235'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' >= TIMESTAMP '2020-05-01 12:34:56.12346'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' >= TIMESTAMP '2020-05-01 12:34:56.123457'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' >= TIMESTAMP '2020-05-01 12:34:56.1234568'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' >= TIMESTAMP '2020-05-01 12:34:56.12345679'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' >= TIMESTAMP '2020-05-01 12:34:56.1234567891'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' >= TIMESTAMP '2020-05-01 12:34:56.1234567891'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' >= TIMESTAMP '2020-05-01 12:34:56.12345678902'")).isEqualTo(false); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' >= TIMESTAMP '2020-05-01 12:34:56.123456789013'")).isEqualTo(false); + } + + @Test + public void testBetween() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' BETWEEN TIMESTAMP '2020-05-01 12:34:55' and TIMESTAMP '2020-05-01 12:34:57'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' BETWEEN TIMESTAMP '2020-05-01 12:34:56.0' and TIMESTAMP '2020-05-01 12:34:56.2'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' BETWEEN TIMESTAMP '2020-05-01 12:34:56.11' and TIMESTAMP '2020-05-01 12:34:56.13'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' BETWEEN TIMESTAMP '2020-05-01 12:34:56.122' and TIMESTAMP '2020-05-01 12:34:56.124'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' BETWEEN TIMESTAMP '2020-05-01 12:34:56.1233' and TIMESTAMP '2020-05-01 12:34:56.1235'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' BETWEEN TIMESTAMP '2020-05-01 12:34:56.12344' and TIMESTAMP '2020-05-01 12:34:56.12346'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' BETWEEN TIMESTAMP '2020-05-01 12:34:56.123455' and TIMESTAMP '2020-05-01 12:34:56.123457'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' BETWEEN TIMESTAMP '2020-05-01 12:34:56.1234566' and TIMESTAMP '2020-05-01 12:34:56.1234568'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' BETWEEN TIMESTAMP '2020-05-01 12:34:56.12345677' and TIMESTAMP '2020-05-01 12:34:56.12345679'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' BETWEEN TIMESTAMP '2020-05-01 12:34:56.123456788' and TIMESTAMP '2020-05-01 12:34:56.123456790'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' BETWEEN TIMESTAMP '2020-05-01 12:34:56.1234567889' and TIMESTAMP '2020-05-01 12:34:56.1234567891'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' BETWEEN TIMESTAMP '2020-05-01 12:34:56.1234567890' and TIMESTAMP '2020-05-01 12:34:56.12345678902'")).isEqualTo(true); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' BETWEEN TIMESTAMP '2020-05-01 12:34:56.123456789011' and TIMESTAMP '2020-05-01 12:34:56.123456789013'")).isEqualTo(true); + } + + @Test + public void testAddIntervalDayToSecond() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.123'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.223'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.243'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.246'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.2464'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.24645'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.246456'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.2464567'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.24645678'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.246456789'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.2464567890'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.24645678901'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' + INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:57.246456789012'"); + + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56'")).matches("TIMESTAMP '2020-05-01 12:34:57.123'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.1'")).matches("TIMESTAMP '2020-05-01 12:34:57.223'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.12'")).matches("TIMESTAMP '2020-05-01 12:34:57.243'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.123'")).matches("TIMESTAMP '2020-05-01 12:34:57.246'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.1234'")).matches("TIMESTAMP '2020-05-01 12:34:57.2464'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.12345'")).matches("TIMESTAMP '2020-05-01 12:34:57.24645'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.123456'")).matches("TIMESTAMP '2020-05-01 12:34:57.246456'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.1234567'")).matches("TIMESTAMP '2020-05-01 12:34:57.2464567'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.12345678'")).matches("TIMESTAMP '2020-05-01 12:34:57.24645678'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.123456789'")).matches("TIMESTAMP '2020-05-01 12:34:57.246456789'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.1234567890'")).matches("TIMESTAMP '2020-05-01 12:34:57.2464567890'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.12345678901'")).matches("TIMESTAMP '2020-05-01 12:34:57.24645678901'"); + assertThat(assertions.expression("INTERVAL '1.123' SECOND + TIMESTAMP '2020-05-01 12:34:56.123456789012'")).matches("TIMESTAMP '2020-05-01 12:34:57.246456789012'"); + + // interval is currently p = 3, so the timestamp(p) + interval day to second yields timestamp(3) for p = [0, 1, 2, 3] + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.000'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.100'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.120'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.123'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.1234'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.12345'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.123456'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.1234567'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.12345678'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.123456789'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.1234567890'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.12345678901'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' + INTERVAL '1' DAY")).matches("TIMESTAMP '2020-05-02 12:34:56.123456789012'"); + + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56'")).matches("TIMESTAMP '2020-05-02 12:34:56.000'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.1'")).matches("TIMESTAMP '2020-05-02 12:34:56.100'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.12'")).matches("TIMESTAMP '2020-05-02 12:34:56.120'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.123'")).matches("TIMESTAMP '2020-05-02 12:34:56.123'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.1234'")).matches("TIMESTAMP '2020-05-02 12:34:56.1234'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.12345'")).matches("TIMESTAMP '2020-05-02 12:34:56.12345'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.123456'")).matches("TIMESTAMP '2020-05-02 12:34:56.123456'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.1234567'")).matches("TIMESTAMP '2020-05-02 12:34:56.1234567'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.12345678'")).matches("TIMESTAMP '2020-05-02 12:34:56.12345678'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.123456789'")).matches("TIMESTAMP '2020-05-02 12:34:56.123456789'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.1234567890'")).matches("TIMESTAMP '2020-05-02 12:34:56.1234567890'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.12345678901'")).matches("TIMESTAMP '2020-05-02 12:34:56.12345678901'"); + assertThat(assertions.expression("INTERVAL '1' DAY + TIMESTAMP '2020-05-01 12:34:56.123456789012'")).matches("TIMESTAMP '2020-05-02 12:34:56.123456789012'"); + } + + @Test + public void testSubtractIntervalDayToSecond() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:54.877'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:54.977'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:54.997'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.000'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.0004'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.00045'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.000456'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.0004567'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.00045678'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.000456789'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.0004567890'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.00045678901'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' - INTERVAL '1.123' SECOND")).matches("TIMESTAMP '2020-05-01 12:34:55.000456789012'"); + + // interval is currently p = 3, so the timestamp(p) - interval day to second yields timestamp(3) for p = [0, 1, 2, 3] + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.000'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.100'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.120'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.123'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.1234'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.12345'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.123456'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.1234567'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.12345678'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.123456789'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.1234567890'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.12345678901'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' - INTERVAL '1' DAY")).matches("TIMESTAMP '2020-04-30 12:34:56.123456789012'"); + } + + @Test + public void testAddIntervalYearToMonth() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.1'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.12'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.123'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.1234'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.12345'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.123456'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.1234567'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.12345678'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.123456789'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.1234567890'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.12345678901'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' + INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-06-01 12:34:56.123456789012'"); + + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56'")).matches("TIMESTAMP '2020-06-01 12:34:56'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.1'")).matches("TIMESTAMP '2020-06-01 12:34:56.1'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.12'")).matches("TIMESTAMP '2020-06-01 12:34:56.12'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.123'")).matches("TIMESTAMP '2020-06-01 12:34:56.123'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.1234'")).matches("TIMESTAMP '2020-06-01 12:34:56.1234'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.12345'")).matches("TIMESTAMP '2020-06-01 12:34:56.12345'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.123456'")).matches("TIMESTAMP '2020-06-01 12:34:56.123456'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.1234567'")).matches("TIMESTAMP '2020-06-01 12:34:56.1234567'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.12345678'")).matches("TIMESTAMP '2020-06-01 12:34:56.12345678'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.123456789'")).matches("TIMESTAMP '2020-06-01 12:34:56.123456789'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.1234567890'")).matches("TIMESTAMP '2020-06-01 12:34:56.1234567890'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.12345678901'")).matches("TIMESTAMP '2020-06-01 12:34:56.12345678901'"); + assertThat(assertions.expression("INTERVAL '1' MONTH + TIMESTAMP '2020-05-01 12:34:56.123456789012'")).matches("TIMESTAMP '2020-06-01 12:34:56.123456789012'"); + } + + @Test + public void testSubtractIntervalYearToMonth() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.1'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.12'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.123'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.1234'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.12345'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.123456'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.1234567'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.12345678'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.123456789'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.1234567890'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.12345678901'"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012' - INTERVAL '1' MONTH")).matches("TIMESTAMP '2020-04-01 12:34:56.123456789012'"); + } + + @Test + public void testSubtract() + { + // round down + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56' - TIMESTAMP '2020-05-01 12:34:55'")).matches("INTERVAL '1' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.2' - TIMESTAMP '2020-05-01 12:34:55.1'")).matches("INTERVAL '1.1' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.22' - TIMESTAMP '2020-05-01 12:34:55.11'")).matches("INTERVAL '1.11' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.222' - TIMESTAMP '2020-05-01 12:34:55.111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.2222' - TIMESTAMP '2020-05-01 12:34:55.1111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.22222' - TIMESTAMP '2020-05-01 12:34:55.11111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.222222' - TIMESTAMP '2020-05-01 12:34:55.111111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.2222222' - TIMESTAMP '2020-05-01 12:34:55.1111111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.22222222' - TIMESTAMP '2020-05-01 12:34:55.11111111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.222222222' - TIMESTAMP '2020-05-01 12:34:55.111111111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.2222222222' - TIMESTAMP '2020-05-01 12:34:55.1111111111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.22222222222' - TIMESTAMP '2020-05-01 12:34:55.11111111111'")).matches("INTERVAL '1.111' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.222222222222' - TIMESTAMP '2020-05-01 12:34:55.111111111111'")).matches("INTERVAL '1.111' SECOND"); + + // round up + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.9' - TIMESTAMP '2020-05-01 12:34:55.1'")).matches("INTERVAL '1.8' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.99' - TIMESTAMP '2020-05-01 12:34:55.11'")).matches("INTERVAL '1.88' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.999' - TIMESTAMP '2020-05-01 12:34:55.111'")).matches("INTERVAL '1.888' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.9999' - TIMESTAMP '2020-05-01 12:34:55.1111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.99999' - TIMESTAMP '2020-05-01 12:34:55.11111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.999999' - TIMESTAMP '2020-05-01 12:34:55.111111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.9999999' - TIMESTAMP '2020-05-01 12:34:55.1111111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.99999999' - TIMESTAMP '2020-05-01 12:34:55.11111111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.999999999' - TIMESTAMP '2020-05-01 12:34:55.111111111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.9999999999' - TIMESTAMP '2020-05-01 12:34:55.1111111111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.99999999999' - TIMESTAMP '2020-05-01 12:34:55.11111111111'")).matches("INTERVAL '1.889' SECOND"); + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.999999999999' - TIMESTAMP '2020-05-01 12:34:55.111111111111'")).matches("INTERVAL '1.889' SECOND"); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestTimestamp.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestTimestamp.java new file mode 100644 index 0000000000000..89ab141559aea --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/BaseTestTimestamp.java @@ -0,0 +1,1501 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import io.prestosql.Session; +import io.prestosql.SystemSessionProperties; +import io.prestosql.spi.type.SqlTimestamp; +import io.prestosql.spi.type.TimeZoneKey; +import io.prestosql.sql.query.QueryAssertions; +import io.prestosql.testing.QueryRunner; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.util.function.BiFunction; + +import static io.prestosql.spi.type.TimestampType.createTimestampType; +import static io.prestosql.spi.type.VarcharType.createVarcharType; +import static io.prestosql.testing.TestingSession.DEFAULT_TIME_ZONE_KEY; +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static io.prestosql.type.Timestamps.MICROSECONDS_PER_SECOND; +import static io.prestosql.type.Timestamps.PICOSECONDS_PER_MICROSECOND; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public abstract class BaseTestTimestamp +{ + private static final TimeZoneKey SESSION_TIME_ZONE = DEFAULT_TIME_ZONE_KEY; + + private final boolean legacyTimestamp; + protected QueryAssertions assertions; + + protected BaseTestTimestamp(boolean legacyTimestamp) + { + this.legacyTimestamp = legacyTimestamp; + } + + @BeforeClass + public void init() + { + Session session = testSessionBuilder() + .setSystemProperty("legacy_timestamp", String.valueOf(legacyTimestamp)) + .setTimeZoneKey(SESSION_TIME_ZONE) + .build(); + assertions = new QueryAssertions(session); + } + + @AfterClass(alwaysRun = true) + public void teardown() + { + assertions.close(); + assertions = null; + } + + @Test + public void testLiterals() + { + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56'")) + .hasType(createTimestampType(0)) + .isEqualTo(timestamp(0, 2020, 5, 1, 12, 34, 56, 0)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1'")) + .hasType(createTimestampType(1)) + .isEqualTo(timestamp(1, 2020, 5, 1, 12, 34, 56, 100_000_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12'")) + .hasType(createTimestampType(2)) + .isEqualTo(timestamp(2, 2020, 5, 1, 12, 34, 56, 120_000_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123'")) + .hasType(createTimestampType(3)) + .isEqualTo(timestamp(3, 2020, 5, 1, 12, 34, 56, 123_000_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234'")) + .hasType(createTimestampType(4)) + .isEqualTo(timestamp(4, 2020, 5, 1, 12, 34, 56, 123_400_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345'")) + .hasType(createTimestampType(5)) + .isEqualTo(timestamp(5, 2020, 5, 1, 12, 34, 56, 123_450_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456'")) + .hasType(createTimestampType(6)) + .isEqualTo(timestamp(6, 2020, 5, 1, 12, 34, 56, 123_456_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567'")) + .hasType(createTimestampType(7)) + .isEqualTo(timestamp(7, 2020, 5, 1, 12, 34, 56, 123_456_700_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678'")) + .hasType(createTimestampType(8)) + .isEqualTo(timestamp(8, 2020, 5, 1, 12, 34, 56, 123_456_780_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789'")) + .hasType(createTimestampType(9)) + .isEqualTo(timestamp(9, 2020, 5, 1, 12, 34, 56, 123_456_789_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890'")) + .hasType(createTimestampType(10)) + .isEqualTo(timestamp(10, 2020, 5, 1, 12, 34, 56, 123_456_789_000L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.12345678901'")) + .hasType(createTimestampType(11)) + .isEqualTo(timestamp(11, 2020, 5, 1, 12, 34, 56, 123_456_789_010L)); + + assertThat(assertions.expression("TIMESTAMP '2020-05-01 12:34:56.123456789012'")) + .hasType(createTimestampType(12)) + .isEqualTo(timestamp(12, 2020, 5, 1, 12, 34, 56, 123_456_789_012L)); + + assertThatThrownBy(() -> assertions.expression("TIMESTAMP '2020-05-01 12:34:56.1234567890123'")) + .hasMessage("line 1:8: TIMESTAMP precision must be in range [0, 12]"); + + assertThatThrownBy(() -> assertions.expression("TIMESTAMP '2020-13-01'")) + .hasMessage("line 1:8: '2020-13-01' is not a valid timestamp literal"); + + assertThatThrownBy(() -> assertions.expression("TIMESTAMP 'xxx'")) + .hasMessage("line 1:8: 'xxx' is not a valid timestamp literal"); + + // negative epoch + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56'")) + .isEqualTo(timestamp(0, 1500, 5, 1, 12, 34, 56, 0)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.1'")) + .isEqualTo(timestamp(1, 1500, 5, 1, 12, 34, 56, 100_000_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.12'")) + .isEqualTo(timestamp(2, 1500, 5, 1, 12, 34, 56, 120_000_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.123'")) + .isEqualTo(timestamp(3, 1500, 5, 1, 12, 34, 56, 123_000_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.1234'")) + .isEqualTo(timestamp(4, 1500, 5, 1, 12, 34, 56, 123_400_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.12345'")) + .isEqualTo(timestamp(5, 1500, 5, 1, 12, 34, 56, 123_450_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.123456'")) + .isEqualTo(timestamp(6, 1500, 5, 1, 12, 34, 56, 123_456_000_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.1234567'")) + .isEqualTo(timestamp(7, 1500, 5, 1, 12, 34, 56, 123_456_700_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.12345678'")) + .isEqualTo(timestamp(8, 1500, 5, 1, 12, 34, 56, 123_456_780_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.123456789'")) + .isEqualTo(timestamp(9, 1500, 5, 1, 12, 34, 56, 123_456_789_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.1234567890'")) + .isEqualTo(timestamp(10, 1500, 5, 1, 12, 34, 56, 123_456_789_000L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.12345678901'")) + .isEqualTo(timestamp(11, 1500, 5, 1, 12, 34, 56, 123_456_789_010L)); + + assertThat(assertions.expression("TIMESTAMP '1500-05-01 12:34:56.123456789012'")) + .isEqualTo(timestamp(12, 1500, 5, 1, 12, 34, 56, 123_456_789_012L)); + } + + @Test + public void testLocalTimestamp() + { + // round down + Session session = assertions.sessionBuilder() + .setStart(Instant.from(ZonedDateTime.of(2020, 5, 1, 12, 34, 56, 111111111, assertions.getDefaultSession().getTimeZoneKey().getZoneId()))) + .build(); + + assertThat(assertions.expression("localtimestamp(0)", session)).matches("TIMESTAMP '2020-05-01 12:34:56'"); + assertThat(assertions.expression("localtimestamp(1)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1'"); + assertThat(assertions.expression("localtimestamp(2)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.11'"); + assertThat(assertions.expression("localtimestamp(3)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111'"); + assertThat(assertions.expression("localtimestamp(4)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1111'"); + assertThat(assertions.expression("localtimestamp(5)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.11111'"); + assertThat(assertions.expression("localtimestamp(6)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111111'"); + assertThat(assertions.expression("localtimestamp(7)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1111111'"); + assertThat(assertions.expression("localtimestamp(8)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.11111111'"); + assertThat(assertions.expression("localtimestamp(9)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111111111'"); + assertThat(assertions.expression("localtimestamp(10)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1111111110'"); // Java instant provides p = 9 precision + assertThat(assertions.expression("localtimestamp(11)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.11111111100'"); // Java instant provides p = 9 precision + assertThat(assertions.expression("localtimestamp(12)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111111111000'"); // Java instant provides p = 9 precision + + // round up + session = assertions.sessionBuilder() + .setStart(Instant.from(ZonedDateTime.of(2020, 5, 1, 12, 34, 56, 555555555, assertions.getDefaultSession().getTimeZoneKey().getZoneId()))) + .build(); + + assertThat(assertions.expression("localtimestamp(0)", session)).matches("TIMESTAMP '2020-05-01 12:34:57'"); + assertThat(assertions.expression("localtimestamp(1)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.6'"); + assertThat(assertions.expression("localtimestamp(2)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.56'"); + assertThat(assertions.expression("localtimestamp(3)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556'"); + assertThat(assertions.expression("localtimestamp(4)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.5556'"); + assertThat(assertions.expression("localtimestamp(5)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.55556'"); + assertThat(assertions.expression("localtimestamp(6)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.555556'"); + assertThat(assertions.expression("localtimestamp(7)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.5555556'"); + assertThat(assertions.expression("localtimestamp(8)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.55555556'"); + assertThat(assertions.expression("localtimestamp(9)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.555555555'"); + assertThat(assertions.expression("localtimestamp(10)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.5555555550'"); // Java instant provides p = 9 precision + assertThat(assertions.expression("localtimestamp(11)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.55555555500'"); // Java instant provides p = 9 precision + assertThat(assertions.expression("localtimestamp(12)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.555555555000'"); // Java instant provides p = 9 precision + } + + @Test + public void testCastToDate() + { + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234567' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345678' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456789' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234567890' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345678901' AS DATE)")).matches("DATE '2020-05-01'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456789012' AS DATE)")).matches("DATE '2020-05-01'"); + } + + @Test + public void testCastFromDate() + { + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-01 00:00:00'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-01 00:00:00.0'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-01 00:00:00.00'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-01 00:00:00.000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-01 00:00:00.0000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-01 00:00:00.00000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-01 00:00:00.000000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-01 00:00:00.0000000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-01 00:00:00.00000000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-01 00:00:00.000000000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-01 00:00:00.0000000000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(11))")).matches("TIMESTAMP '2020-05-01 00:00:00.00000000000'"); + assertThat(assertions.expression("CAST(DATE '2020-05-01' AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-01 00:00:00.000000000000'"); + } + + @Test + public void testCastToTime() + { + // round down + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56' AS TIME)")).matches("TIME '12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1' AS TIME)")).matches("TIME '12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11' AS TIME)")).matches("TIME '12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1111111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11111111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111111111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1111111111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11111111111' AS TIME)")).matches("TIME '12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111111111111' AS TIME)")).matches("TIME '12:34:56.111'"); + + // round up + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56' AS TIME)")).matches("TIME '12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5' AS TIME)")).matches("TIME '12:34:56.5'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55' AS TIME)")).matches("TIME '12:34:56.55'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555' AS TIME)")).matches("TIME '12:34:56.555'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5555555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55555555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555555555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5555555555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55555555555' AS TIME)")).matches("TIME '12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555555555555' AS TIME)")).matches("TIME '12:34:56.556'"); + } + + @Test + public void testCastFromTime() + { + Session session = assertions.sessionBuilder() + .setStart(Instant.from(ZonedDateTime.of(2020, 5, 1, 12, 34, 56, 123456789, assertions.getDefaultSession().getTimeZoneKey().getZoneId()))) + .build(); + + // TODO: date part should be 2020-05-01. See https://github.com/prestosql/presto/issues/3845 + + // round down + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(0))", session)).matches("TIMESTAMP '1970-01-01 12:34:56'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(1))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(2))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(3))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(4))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1110'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(5))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11100'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(6))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(7))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1110000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(8))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11100000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(9))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(10))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1110000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(11))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11100000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111' AS TIMESTAMP(12))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111000000000'"); + + // round up + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(0))", session)).matches("TIMESTAMP '1970-01-01 12:34:57'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(1))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(2))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(3))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(4))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.5550'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(5))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.55500'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(6))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(7))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.5550000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(8))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.55500000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(9))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(10))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.5550000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(11))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.55500000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555' AS TIMESTAMP(12))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555000000000'"); + } + + @Test + public void testCastToTimeWithTimeZone() + { + Session session = assertions.sessionBuilder() + .setTimeZoneKey(TimeZoneKey.getTimeZoneKeyForOffset(-5 * 60)) + .build(); + + // Should be equivalent to CAST(CAST(x AS TIMESTAMP(p) WITH TIME ZONE) AS TIME(p) WITH TIME ZONE) + + // round down + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.1 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.11 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1111111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11111111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111111111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.1111111111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.11111111111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.111111111111' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.111 -05:00'"); + + // round up + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.5 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.55 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.555 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5555555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55555555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555555555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.5555555555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.55555555555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2001-1-22 12:34:56.555555555555' AS TIME WITH TIME ZONE)", session)).matches("TIME '12:34:56.556 -05:00'"); + } + + @Test + public void testCastFromTimeWithTimeZone() + { + Session session = assertions.sessionBuilder() + .setTimeZoneKey(TimeZoneKey.getTimeZoneKeyForOffset(-8 * 60)) + .build(); + + // should be equivalent to CAST(CAST(x, 'America/Los_Angeles') AS TIMESTAMP WITHOUT TIME ZONE) + // TODO: date part should be 2020-05-01. See https://github.com/prestosql/presto/issues/3845 + + // round down + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(0))", session)).matches("TIMESTAMP '1970-01-01 12:34:56'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(1))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(2))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(3))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(4))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1110'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(5))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11100'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(6))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(7))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1110000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(8))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11100000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(9))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(10))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.1110000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(11))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.11100000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.111 -08:00' AS TIMESTAMP(12))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.111000000000'"); + + // round up + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(0))", session)).matches("TIMESTAMP '1970-01-01 12:34:57'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(1))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(2))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(3))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(4))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.5550'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(5))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.55500'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(6))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(7))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.5550000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(8))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.55500000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(9))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(10))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.5550000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(11))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.55500000000'"); + assertThat(assertions.expression("CAST(TIME '12:34:56.555 -08:00' AS TIMESTAMP(12))", session)).matches("TIMESTAMP '1970-01-01 12:34:56.555000000000'"); + } + + @Test + public void testCastToTimestampWithTimeZone() + { + Session session = assertions.sessionBuilder() + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey("America/Los_Angeles")) + .build(); + + // round down + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.11' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.11 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.11111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1111111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.11111111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111111111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1111111111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.11111111111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111111111111' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + + // round up + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.5' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.5 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.55' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.55 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.555 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.5555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.55555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.5555555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.55555555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555555555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.5555555555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.55555555555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555555555555' AS TIMESTAMP WITH TIME ZONE)", session)).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + } + + @Test + public void testCastFromTimestampWithTimeZone() + { + Session session = assertions.sessionBuilder() + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey("America/Los_Angeles")) + .build(); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56 America/Los_Angeles' AS TIMESTAMP(0))", session)).matches("TIMESTAMP '2020-05-01 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1 America/Los_Angeles' AS TIMESTAMP(1))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12 America/Los_Angeles' AS TIMESTAMP(2))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.12'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(3))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.123'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(4))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1230'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(5))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.12300'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(6))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.123000'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(7))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1230000'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(8))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.12300000'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(9))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.123000000'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(10))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1230000000'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(11))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.12300000000'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123 America/Los_Angeles' AS TIMESTAMP(12))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.123000000000'"); + + // round down + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles' AS TIMESTAMP(0))", session)).matches("TIMESTAMP '2020-05-01 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles' AS TIMESTAMP(1))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles' AS TIMESTAMP(2))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles' AS TIMESTAMP(3))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.111'"); + + // round up + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555 America/Los_Angeles' AS TIMESTAMP(0))", session)).matches("TIMESTAMP '2020-05-01 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555 America/Los_Angeles' AS TIMESTAMP(1))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555 America/Los_Angeles' AS TIMESTAMP(2))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.555 America/Los_Angeles' AS TIMESTAMP(3))", session)).matches("TIMESTAMP '2020-05-01 12:34:56.555'"); + } + + @Test + public void testCastToJson() + { + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.1\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.12\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.123\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.1234\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.12345\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.123456\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234567' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.1234567\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345678' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.12345678\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456789' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.123456789\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234567890' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.1234567890\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345678901' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.12345678901\"'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456789012' AS JSON)")).matches("JSON '\"2020-05-01 12:34:56.123456789012\"'"); + } + + @Test + public void testCastToVarchar() + { + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.1"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.12"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.123"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.1234"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.12345"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.123456"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234567' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.1234567"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345678' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.12345678"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456789' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.123456789"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.1234567890' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.1234567890"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.12345678901' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.12345678901"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-01 12:34:56.123456789012' AS VARCHAR)")).isEqualTo("2020-05-01 12:34:56.123456789012"); + + // negative epoch + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.1' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.1"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.12' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.12"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.123' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.123"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.1234' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.1234"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.12345' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.12345"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.123456' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.123456"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.1234567' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.1234567"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.12345678' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.12345678"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.123456789' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.123456789"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.1234567890' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.1234567890"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.12345678901' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.12345678901"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-01 12:34:56.123456789012' AS VARCHAR)")).isEqualTo("1500-05-01 12:34:56.123456789012"); + } + + @Test + public void testCastFromVarchar() + { + // round down + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-01 12:34:56'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-01 12:34:56.1'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-01 12:34:56.11'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-01 12:34:56.111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-01 12:34:56.1111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-01 12:34:56.11111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-01 12:34:56.111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-01 12:34:56.1111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-01 12:34:56.11111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-01 12:34:56.111111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-01 12:34:56.1111111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(11))")).matches("TIMESTAMP '2020-05-01 12:34:56.11111111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-01 12:34:56.111111111111'"); + + // round up + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-01 12:34:57'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-01 12:34:56.6'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-01 12:34:56.56'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-01 12:34:56.556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-01 12:34:56.5556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-01 12:34:56.55556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-01 12:34:56.555556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-01 12:34:56.5555556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-01 12:34:56.55555556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-01 12:34:56.555555556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-01 12:34:56.5555555556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(11))")).matches("TIMESTAMP '2020-05-01 12:34:56.55555555556'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.555555555555' AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-01 12:34:56.555555555555'"); + + // negative epoch, round down + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-01 12:34:56'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-01 12:34:56.1'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-01 12:34:56.11'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-01 12:34:56.111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-01 12:34:56.1111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-01 12:34:56.11111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-01 12:34:56.111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-01 12:34:56.1111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-01 12:34:56.11111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-01 12:34:56.111111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-01 12:34:56.1111111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(11))")).matches("TIMESTAMP '2020-05-01 12:34:56.11111111111'"); + assertThat(assertions.expression("CAST('2020-05-01 12:34:56.111111111111' AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-01 12:34:56.111111111111'"); + + // negative epoch, round up + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-01 12:34:57'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-01 12:34:56.6'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-01 12:34:56.56'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-01 12:34:56.556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-01 12:34:56.5556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-01 12:34:56.55556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-01 12:34:56.555556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-01 12:34:56.5555556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-01 12:34:56.55555556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '1500-05-01 12:34:56.555555556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(10))")).matches("TIMESTAMP '1500-05-01 12:34:56.5555555556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(11))")).matches("TIMESTAMP '1500-05-01 12:34:56.55555555556'"); + assertThat(assertions.expression("CAST('1500-05-01 12:34:56.555555555555' AS TIMESTAMP(12))")).matches("TIMESTAMP '1500-05-01 12:34:56.555555555555'"); + } + + @Test + public void testLowerDigitsZeroed() + { + // round down + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(0)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:56.000000000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(3)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:56.111000000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(6)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(9)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111111000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(0)) AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(3)) AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(6)) AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.111' AS TIMESTAMP(0)) AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.000'"); + + // round up + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(0)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:57.000000000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(3)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:56.556000000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(6)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:56.555556000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(9)) AS TIMESTAMP(12))")).matches("TIMESTAMP '2020-05-10 12:34:56.555555556000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(0)) AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:57.000000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(3)) AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.556000'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(6)) AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.555555'"); + assertThat(assertions.expression("CAST(CAST(TIMESTAMP '2020-05-10 12:34:56.555' AS TIMESTAMP(0)) AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:57.000'"); + } + + @Test + public void testRoundDown() + { + // positive epoch + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111111' AS TIMESTAMP(11))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111111' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.11111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11111' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.1111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1111' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.111' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.11'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.11' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.1' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + + // positive epoch + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(10))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111111' AS TIMESTAMP(11))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111111' AS TIMESTAMP(10))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111111' AS TIMESTAMP(9))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111111' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111111' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111111' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.111111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111111' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.11111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11111' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.1111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1111' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.111'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.111' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.11'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.11' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.1'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.1' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:56'"); + } + + @Test + public void testRoundUp() + { + // positive epoch + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.55555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-10 12:34:56.555555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-10 12:34:56.5555555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555555' AS TIMESTAMP(11))")).matches("TIMESTAMP '2020-05-10 12:34:56.55555555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.55555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-10 12:34:56.555555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555555' AS TIMESTAMP(10))")).matches("TIMESTAMP '2020-05-10 12:34:56.5555555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.55555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '2020-05-10 12:34:56.555555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '2020-05-10 12:34:56.55555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '2020-05-10 12:34:56.5555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '2020-05-10 12:34:56.555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '2020-05-10 12:34:56.55556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55555' AS TIMESTAMP(4))")).matches("TIMESTAMP '2020-05-10 12:34:56.5556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5555' AS TIMESTAMP(3))")).matches("TIMESTAMP '2020-05-10 12:34:56.556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.555' AS TIMESTAMP(2))")).matches("TIMESTAMP '2020-05-10 12:34:56.56'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.55' AS TIMESTAMP(1))")).matches("TIMESTAMP '2020-05-10 12:34:56.6'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '2020-05-10 12:34:56.5' AS TIMESTAMP(0))")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + + // negative epoch + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.55555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '1500-05-10 12:34:56.555555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(10))")).matches("TIMESTAMP '1500-05-10 12:34:56.5555555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555555' AS TIMESTAMP(11))")).matches("TIMESTAMP '1500-05-10 12:34:56.55555555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.55555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '1500-05-10 12:34:56.555555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555555' AS TIMESTAMP(10))")).matches("TIMESTAMP '1500-05-10 12:34:56.5555555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.55555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555555' AS TIMESTAMP(9))")).matches("TIMESTAMP '1500-05-10 12:34:56.555555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.5555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555555' AS TIMESTAMP(8))")).matches("TIMESTAMP '1500-05-10 12:34:56.55555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.555556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555555' AS TIMESTAMP(7))")).matches("TIMESTAMP '1500-05-10 12:34:56.5555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.55556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555555' AS TIMESTAMP(6))")).matches("TIMESTAMP '1500-05-10 12:34:56.555556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555555' AS TIMESTAMP(5))")).matches("TIMESTAMP '1500-05-10 12:34:56.55556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55555' AS TIMESTAMP(4))")).matches("TIMESTAMP '1500-05-10 12:34:56.5556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5555' AS TIMESTAMP(3))")).matches("TIMESTAMP '1500-05-10 12:34:56.556'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.555' AS TIMESTAMP(2))")).matches("TIMESTAMP '1500-05-10 12:34:56.56'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.55' AS TIMESTAMP(1))")).matches("TIMESTAMP '1500-05-10 12:34:56.6'"); + + assertThat(assertions.expression("CAST(TIMESTAMP '1500-05-10 12:34:56.5' AS TIMESTAMP(0))")).matches("TIMESTAMP '1500-05-10 12:34:57'"); + } + + @Test + public void testToIso8601() + { + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56')")) + .hasType(createVarcharType(22)) + .isEqualTo("2020-05-01T12:34:56"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.1')")) + .hasType(createVarcharType(24)) + .isEqualTo("2020-05-01T12:34:56.1"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.12')")) + .hasType(createVarcharType(25)) + .isEqualTo("2020-05-01T12:34:56.12"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.123')")) + .hasType(createVarcharType(26)) + .isEqualTo("2020-05-01T12:34:56.123"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.1234')")) + .hasType(createVarcharType(27)) + .isEqualTo("2020-05-01T12:34:56.1234"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.12345')")) + .hasType(createVarcharType(28)) + .isEqualTo("2020-05-01T12:34:56.12345"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.123456')")) + .hasType(createVarcharType(29)) + .isEqualTo("2020-05-01T12:34:56.123456"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.1234567')")) + .hasType(createVarcharType(30)) + .isEqualTo("2020-05-01T12:34:56.1234567"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.12345678')")) + .hasType(createVarcharType(31)) + .isEqualTo("2020-05-01T12:34:56.12345678"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.123456789')")) + .hasType(createVarcharType(32)) + .isEqualTo("2020-05-01T12:34:56.123456789"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.1234567890')")) + .hasType(createVarcharType(33)) + .isEqualTo("2020-05-01T12:34:56.1234567890"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.12345678901')")) + .hasType(createVarcharType(34)) + .isEqualTo("2020-05-01T12:34:56.12345678901"); + + assertThat(assertions.expression("to_iso8601(TIMESTAMP '2020-05-01 12:34:56.123456789012')")) + .hasType(createVarcharType(35)) + .isEqualTo("2020-05-01T12:34:56.123456789012"); + } + + @Test + public void testFormat() + { + // round down + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56')")).isEqualTo("2020-05-10T12:34:56"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.1')")).isEqualTo("2020-05-10T12:34:56.100"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.11')")).isEqualTo("2020-05-10T12:34:56.110"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.111')")).isEqualTo("2020-05-10T12:34:56.111"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.1111')")).isEqualTo("2020-05-10T12:34:56.111100"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.11111')")).isEqualTo("2020-05-10T12:34:56.111110"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.111111')")).isEqualTo("2020-05-10T12:34:56.111111"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.1111111')")).isEqualTo("2020-05-10T12:34:56.111111100"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.11111111')")).isEqualTo("2020-05-10T12:34:56.111111110"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.111111111')")).isEqualTo("2020-05-10T12:34:56.111111111"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.1111111111')")).isEqualTo("2020-05-10T12:34:56.111111111"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.11111111111')")).isEqualTo("2020-05-10T12:34:56.111111111"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.111111111111')")).isEqualTo("2020-05-10T12:34:56.111111111"); + + // round up + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56')")).isEqualTo("2020-05-10T12:34:56"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.5')")).isEqualTo("2020-05-10T12:34:56.500"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.55')")).isEqualTo("2020-05-10T12:34:56.550"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.555')")).isEqualTo("2020-05-10T12:34:56.555"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.5555')")).isEqualTo("2020-05-10T12:34:56.555500"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.55555')")).isEqualTo("2020-05-10T12:34:56.555550"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.555555')")).isEqualTo("2020-05-10T12:34:56.555555"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.5555555')")).isEqualTo("2020-05-10T12:34:56.555555500"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.55555555')")).isEqualTo("2020-05-10T12:34:56.555555550"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.555555555')")).isEqualTo("2020-05-10T12:34:56.555555555"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.5555555555')")).isEqualTo("2020-05-10T12:34:56.555555556"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.55555555555')")).isEqualTo("2020-05-10T12:34:56.555555556"); + assertThat(assertions.expression("format('%s', TIMESTAMP '2020-05-10 12:34:56.555555555555')")).isEqualTo("2020-05-10T12:34:56.555555556"); + } + + @Test + public void testFormatDateTime() + { + // format_datetime supports up to millisecond precision + + // round down + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.000"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.1', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.100"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.11', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.110"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.1111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.11111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.111111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.1111111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.11111111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.111111111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.1111111111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.11111111111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.111111111111', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.111"); + + // round up + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.000"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.5', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.500"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.55', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.550"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.555"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.5555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.55555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.555555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.5555555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.55555555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.555555555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.5555555555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.55555555555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + assertThat(assertions.expression("format_datetime(TIMESTAMP '2020-05-10 12:34:56.555555555555', 'yyyy-MM-dd HH:mm:ss.SSS')")).isEqualTo("2020-05-10 12:34:56.556"); + } + + @Test + public void testDateFormat() + { + // date_format supports up to millisecond precision + + // round down + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.000000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.1', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.100000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.11', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.110000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.1111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.11111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.111111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.1111111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.11111111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.111111111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.1111111111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.11111111111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.111111111111', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.111000"); + + // round up + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.000000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.5', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.500000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.55', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.550000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.555000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.5555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.55555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.555555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.5555555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.55555555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.555555555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.5555555555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.55555555555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + assertThat(assertions.expression("date_format(TIMESTAMP '2020-05-10 12:34:56.555555555555', '%Y-%m-%d %H:%i:%s.%f')")).isEqualTo("2020-05-10 12:34:56.556000"); + } + + @Test + public void testWithTimeZone() + { + // round down + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.1', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.1 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.11', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.11 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.1111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.11111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.111111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.1111111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.11111111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.111111111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.1111111111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.11111111111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.111111111111', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.111 America/Los_Angeles'"); + + // round up + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.5', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.5 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.55', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.55 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.555 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.5555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.55555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.555555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.5555555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.55555555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.555555555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.5555555555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.55555555555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + assertThat(assertions.expression("with_timezone(TIMESTAMP '2020-05-01 12:34:56.555555555555', 'America/Los_Angeles')")).matches("TIMESTAMP '2020-05-01 12:34:56.556 America/Los_Angeles'"); + } + + @Test + public void testSequenceIntervalDayToSecond() + { + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56', TIMESTAMP '2020-01-01 12:34:59', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56', " + + "TIMESTAMP '2020-01-01 12:34:57', " + + "TIMESTAMP '2020-01-01 12:34:58', " + + "TIMESTAMP '2020-01-01 12:34:59']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1', TIMESTAMP '2020-01-01 12:34:59.1', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1', " + + "TIMESTAMP '2020-01-01 12:34:57.1', " + + "TIMESTAMP '2020-01-01 12:34:58.1', " + + "TIMESTAMP '2020-01-01 12:34:59.1']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12', TIMESTAMP '2020-01-01 12:34:59.12', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12', " + + "TIMESTAMP '2020-01-01 12:34:57.12', " + + "TIMESTAMP '2020-01-01 12:34:58.12', " + + "TIMESTAMP '2020-01-01 12:34:59.12']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123', TIMESTAMP '2020-01-01 12:34:59.123', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123', " + + "TIMESTAMP '2020-01-01 12:34:57.123', " + + "TIMESTAMP '2020-01-01 12:34:58.123', " + + "TIMESTAMP '2020-01-01 12:34:59.123']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1234', TIMESTAMP '2020-01-01 12:34:59.1234', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1234', " + + "TIMESTAMP '2020-01-01 12:34:57.1234', " + + "TIMESTAMP '2020-01-01 12:34:58.1234', " + + "TIMESTAMP '2020-01-01 12:34:59.1234']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12345', TIMESTAMP '2020-01-01 12:34:59.12345', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12345', " + + "TIMESTAMP '2020-01-01 12:34:57.12345', " + + "TIMESTAMP '2020-01-01 12:34:58.12345', " + + "TIMESTAMP '2020-01-01 12:34:59.12345']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123456', TIMESTAMP '2020-01-01 12:34:59.123456', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123456', " + + "TIMESTAMP '2020-01-01 12:34:57.123456', " + + "TIMESTAMP '2020-01-01 12:34:58.123456', " + + "TIMESTAMP '2020-01-01 12:34:59.123456']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1234567', TIMESTAMP '2020-01-01 12:34:59.1234567', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1234567', " + + "TIMESTAMP '2020-01-01 12:34:57.1234567', " + + "TIMESTAMP '2020-01-01 12:34:58.1234567', " + + "TIMESTAMP '2020-01-01 12:34:59.1234567']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12345678', TIMESTAMP '2020-01-01 12:34:59.12345678', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12345678', " + + "TIMESTAMP '2020-01-01 12:34:57.12345678', " + + "TIMESTAMP '2020-01-01 12:34:58.12345678', " + + "TIMESTAMP '2020-01-01 12:34:59.12345678']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123456789', TIMESTAMP '2020-01-01 12:34:59.123456789', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123456789', " + + "TIMESTAMP '2020-01-01 12:34:57.123456789', " + + "TIMESTAMP '2020-01-01 12:34:58.123456789', " + + "TIMESTAMP '2020-01-01 12:34:59.123456789']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1234567890', TIMESTAMP '2020-01-01 12:34:59.1234567890', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1234567890', " + + "TIMESTAMP '2020-01-01 12:34:57.1234567890', " + + "TIMESTAMP '2020-01-01 12:34:58.1234567890', " + + "TIMESTAMP '2020-01-01 12:34:59.1234567890']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12345678901', TIMESTAMP '2020-01-01 12:34:59.12345678901', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12345678901', " + + "TIMESTAMP '2020-01-01 12:34:57.12345678901', " + + "TIMESTAMP '2020-01-01 12:34:58.12345678901', " + + "TIMESTAMP '2020-01-01 12:34:59.12345678901']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123456789012', TIMESTAMP '2020-01-01 12:34:59.123456789012', INTERVAL '1' SECOND)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123456789012', " + + "TIMESTAMP '2020-01-01 12:34:57.123456789012', " + + "TIMESTAMP '2020-01-01 12:34:58.123456789012', " + + "TIMESTAMP '2020-01-01 12:34:59.123456789012']"); + } + + @Test + public void testSequenceIntervalYearToMonth() + { + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56', TIMESTAMP '2020-04-01 12:34:56', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56', " + + "TIMESTAMP '2020-02-01 12:34:56', " + + "TIMESTAMP '2020-03-01 12:34:56', " + + "TIMESTAMP '2020-04-01 12:34:56']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1', TIMESTAMP '2020-04-01 12:34:56.1', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1', " + + "TIMESTAMP '2020-02-01 12:34:56.1', " + + "TIMESTAMP '2020-03-01 12:34:56.1', " + + "TIMESTAMP '2020-04-01 12:34:56.1']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12', TIMESTAMP '2020-04-01 12:34:59.12', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12', " + + "TIMESTAMP '2020-02-01 12:34:56.12', " + + "TIMESTAMP '2020-03-01 12:34:56.12', " + + "TIMESTAMP '2020-04-01 12:34:56.12']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123', TIMESTAMP '2020-04-01 12:34:59.123', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123', " + + "TIMESTAMP '2020-02-01 12:34:56.123', " + + "TIMESTAMP '2020-03-01 12:34:56.123', " + + "TIMESTAMP '2020-04-01 12:34:56.123']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1234', TIMESTAMP '2020-04-01 12:34:59.1234', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1234', " + + "TIMESTAMP '2020-02-01 12:34:56.1234', " + + "TIMESTAMP '2020-03-01 12:34:56.1234', " + + "TIMESTAMP '2020-04-01 12:34:56.1234']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12345', TIMESTAMP '2020-04-01 12:34:59.12345', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12345', " + + "TIMESTAMP '2020-02-01 12:34:56.12345', " + + "TIMESTAMP '2020-03-01 12:34:56.12345', " + + "TIMESTAMP '2020-04-01 12:34:56.12345']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123456', TIMESTAMP '2020-04-01 12:34:59.123456', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123456', " + + "TIMESTAMP '2020-02-01 12:34:56.123456', " + + "TIMESTAMP '2020-03-01 12:34:56.123456', " + + "TIMESTAMP '2020-04-01 12:34:56.123456']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1234567', TIMESTAMP '2020-04-01 12:34:59.1234567', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1234567', " + + "TIMESTAMP '2020-02-01 12:34:56.1234567', " + + "TIMESTAMP '2020-03-01 12:34:56.1234567', " + + "TIMESTAMP '2020-04-01 12:34:56.1234567']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12345678', TIMESTAMP '2020-04-01 12:34:59.12345678', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12345678', " + + "TIMESTAMP '2020-02-01 12:34:56.12345678', " + + "TIMESTAMP '2020-03-01 12:34:56.12345678', " + + "TIMESTAMP '2020-04-01 12:34:56.12345678']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123456789', TIMESTAMP '2020-04-01 12:34:59.123456789', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123456789', " + + "TIMESTAMP '2020-02-01 12:34:56.123456789', " + + "TIMESTAMP '2020-03-01 12:34:56.123456789', " + + "TIMESTAMP '2020-04-01 12:34:56.123456789']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.1234567890', TIMESTAMP '2020-04-01 12:34:59.1234567890', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.1234567890', " + + "TIMESTAMP '2020-02-01 12:34:56.1234567890', " + + "TIMESTAMP '2020-03-01 12:34:56.1234567890', " + + "TIMESTAMP '2020-04-01 12:34:56.1234567890']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.12345678901', TIMESTAMP '2020-04-01 12:34:59.12345678901', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.12345678901', " + + "TIMESTAMP '2020-02-01 12:34:56.12345678901', " + + "TIMESTAMP '2020-03-01 12:34:56.12345678901', " + + "TIMESTAMP '2020-04-01 12:34:56.12345678901']"); + + assertThat(assertions.expression("sequence(TIMESTAMP '2020-01-01 12:34:56.123456789012', TIMESTAMP '2020-04-01 12:34:59.123456789012', INTERVAL '1' MONTH)")) + .matches("ARRAY[" + + "TIMESTAMP '2020-01-01 12:34:56.123456789012', " + + "TIMESTAMP '2020-02-01 12:34:56.123456789012', " + + "TIMESTAMP '2020-03-01 12:34:56.123456789012', " + + "TIMESTAMP '2020-04-01 12:34:56.123456789012']"); + } + + @Test + public void testDateDiff() + { + // date_diff truncates the fractional part + + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55', TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '1000'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1', TIMESTAMP '2020-05-10 12:34:56.2')")).matches("BIGINT '1100'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11', TIMESTAMP '2020-05-10 12:34:56.22')")).matches("BIGINT '1110'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111', TIMESTAMP '2020-05-10 12:34:56.222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1111', TIMESTAMP '2020-05-10 12:34:56.2222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11111', TIMESTAMP '2020-05-10 12:34:56.22222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111111', TIMESTAMP '2020-05-10 12:34:56.222222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1111111', TIMESTAMP '2020-05-10 12:34:56.2222222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11111111', TIMESTAMP '2020-05-10 12:34:56.22222222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111111111', TIMESTAMP '2020-05-10 12:34:56.222222222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1111111111', TIMESTAMP '2020-05-10 12:34:56.2222222222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11111111111', TIMESTAMP '2020-05-10 12:34:56.22222222222')")).matches("BIGINT '1111'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111111111111', TIMESTAMP '2020-05-10 12:34:56.222222222222')")).matches("BIGINT '1111'"); + + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55', TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '1000'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1', TIMESTAMP '2020-05-10 12:34:56.9')")).matches("BIGINT '1800'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11', TIMESTAMP '2020-05-10 12:34:56.99')")).matches("BIGINT '1880'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111', TIMESTAMP '2020-05-10 12:34:56.999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1111', TIMESTAMP '2020-05-10 12:34:56.9999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11111', TIMESTAMP '2020-05-10 12:34:56.99999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111111', TIMESTAMP '2020-05-10 12:34:56.999999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1111111', TIMESTAMP '2020-05-10 12:34:56.9999999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11111111', TIMESTAMP '2020-05-10 12:34:56.99999999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111111111', TIMESTAMP '2020-05-10 12:34:56.999999999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.1111111111', TIMESTAMP '2020-05-10 12:34:56.9999999999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.11111111111', TIMESTAMP '2020-05-10 12:34:56.99999999999')")).matches("BIGINT '1888'"); + assertThat(assertions.expression("date_diff('millisecond', TIMESTAMP '2020-05-10 12:34:55.111111111111', TIMESTAMP '2020-05-10 12:34:56.999999999999')")).matches("BIGINT '1888'"); + + // coarser unit + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55', TIMESTAMP '2020-05-10 12:34:56')")).matches("BIGINT '1'"); + + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1', TIMESTAMP '2020-05-10 12:34:56.2')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11', TIMESTAMP '2020-05-10 12:34:56.22')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111', TIMESTAMP '2020-05-10 12:34:56.222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1111', TIMESTAMP '2020-05-10 12:34:56.2222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11111', TIMESTAMP '2020-05-10 12:34:56.22222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111111', TIMESTAMP '2020-05-10 12:34:56.222222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1111111', TIMESTAMP '2020-05-10 12:34:56.2222222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11111111', TIMESTAMP '2020-05-10 12:34:56.22222222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111111111', TIMESTAMP '2020-05-10 12:34:56.222222222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1111111111', TIMESTAMP '2020-05-10 12:34:56.2222222222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11111111111', TIMESTAMP '2020-05-10 12:34:56.22222222222')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111111111111', TIMESTAMP '2020-05-10 12:34:56.222222222222')")).matches("BIGINT '1'"); + + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1', TIMESTAMP '2020-05-10 12:34:56.9')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11', TIMESTAMP '2020-05-10 12:34:56.99')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111', TIMESTAMP '2020-05-10 12:34:56.999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1111', TIMESTAMP '2020-05-10 12:34:56.9999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11111', TIMESTAMP '2020-05-10 12:34:56.99999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111111', TIMESTAMP '2020-05-10 12:34:56.999999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1111111', TIMESTAMP '2020-05-10 12:34:56.9999999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11111111', TIMESTAMP '2020-05-10 12:34:56.99999999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111111111', TIMESTAMP '2020-05-10 12:34:56.999999999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.1111111111', TIMESTAMP '2020-05-10 12:34:56.9999999999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.11111111111', TIMESTAMP '2020-05-10 12:34:56.99999999999')")).matches("BIGINT '1'"); + assertThat(assertions.expression("date_diff('hour', TIMESTAMP '2020-05-10 11:34:55.111111111111', TIMESTAMP '2020-05-10 12:34:56.999999999999')")).matches("BIGINT '1'"); + } + + @Test + public void testDateAdd() + { + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-10 12:34:56'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.1')")).matches("TIMESTAMP '2020-05-10 12:34:56.1'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.10')")).matches("TIMESTAMP '2020-05-10 12:34:56.10'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.100')")).matches("TIMESTAMP '2020-05-10 12:34:56.101'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.1000')")).matches("TIMESTAMP '2020-05-10 12:34:56.1010'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.10000')")).matches("TIMESTAMP '2020-05-10 12:34:56.10100'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.100000')")).matches("TIMESTAMP '2020-05-10 12:34:56.101000'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.1000000')")).matches("TIMESTAMP '2020-05-10 12:34:56.1010000'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.10000000')")).matches("TIMESTAMP '2020-05-10 12:34:56.10100000'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.100000000')")).matches("TIMESTAMP '2020-05-10 12:34:56.101000000'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.1000000000')")).matches("TIMESTAMP '2020-05-10 12:34:56.1010000000'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.10000000000')")).matches("TIMESTAMP '2020-05-10 12:34:56.10100000000'"); + assertThat(assertions.expression("date_add('millisecond', 1, TIMESTAMP '2020-05-10 12:34:56.100000000000')")).matches("TIMESTAMP '2020-05-10 12:34:56.101000000000'"); + + assertThat(assertions.expression("date_add('millisecond', 1000, TIMESTAMP '2020-05-10 12:34:56')")).matches("TIMESTAMP '2020-05-10 12:34:57'"); + } + + @Test + public void testLastDayOfMonth() + { + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.1')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.12')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.123')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.1234')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.12345')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.123456')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.1234567')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.12345678')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.123456789')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.1234567891')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.12345678912')")).matches("DATE '2020-05-31'"); + assertThat(assertions.expression("last_day_of_month(TIMESTAMP '2020-05-01 12:34:56.123456789123')")).matches("DATE '2020-05-31'"); + } + + private static BiFunction timestamp(int precision, int year, int month, int day, int hour, int minute, int second, long picoOfSecond) + { + return (session, queryRunner) -> { + LocalDateTime base = LocalDateTime.of(year, month, day, hour, minute, second); + + ZoneOffset offset = ZoneOffset.UTC; + if (SystemSessionProperties.isLegacyTimestamp(session)) { + offset = session.getTimeZoneKey().getZoneId() + .getRules() + .getValidOffsets(base) + .get(0); + } + + long epochMicros = base.toEpochSecond(offset) * MICROSECONDS_PER_SECOND + picoOfSecond / PICOSECONDS_PER_MICROSECOND; + int picosOfMicro = (int) (picoOfSecond % PICOSECONDS_PER_MICROSECOND); + + if (SystemSessionProperties.isLegacyTimestamp(session)) { + return SqlTimestamp.newLegacyInstance(precision, epochMicros, picosOfMicro, session.getTimeZoneKey()); + } + + return SqlTimestamp.newInstance(precision, epochMicros, picosOfMicro); + }; + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestDateTrunc.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestDateTrunc.java new file mode 100644 index 0000000000000..815a0da276ea9 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestDateTrunc.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +public class TestDateTrunc + extends BaseTestDateTrunc +{ + protected TestDateTrunc() + { + super(false); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestDateTruncLegacy.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestDateTruncLegacy.java new file mode 100644 index 0000000000000..ecf38da608a42 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestDateTruncLegacy.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +public class TestDateTruncLegacy + extends BaseTestDateTrunc +{ + public TestDateTruncLegacy() + { + super(true); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestExtract.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestExtract.java new file mode 100644 index 0000000000000..546bb64d0cc4c --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestExtract.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +public class TestExtract + extends BaseTestExtract +{ + public TestExtract() + { + super(false); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestExtractLegacy.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestExtractLegacy.java new file mode 100644 index 0000000000000..2b0c741d20132 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestExtractLegacy.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +public class TestExtractLegacy + extends BaseTestExtract +{ + public TestExtractLegacy() + { + super(true); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestOperators.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestOperators.java new file mode 100644 index 0000000000000..3b784a3bfaf7f --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestOperators.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +public class TestOperators + extends BaseTestOperators +{ + protected TestOperators() + { + super(false); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestOperatorsLegacy.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestOperatorsLegacy.java new file mode 100644 index 0000000000000..dd7c8e37e3972 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestOperatorsLegacy.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +public class TestOperatorsLegacy + extends BaseTestOperators +{ + protected TestOperatorsLegacy() + { + super(true); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestTimestamp.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestTimestamp.java new file mode 100644 index 0000000000000..94659f65de1d2 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestTimestamp.java @@ -0,0 +1,45 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +import org.testng.annotations.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +public class TestTimestamp + extends BaseTestTimestamp +{ + protected TestTimestamp() + { + super(false); + } + + @Test + public void testToUnixTime() + { + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56')")).matches("1589114096e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("1589114096.1e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("1589114096.12e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("1589114096.123e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("1589114096.1234e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("1589114096.12345e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("1589114096.123456e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("1589114096.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("1589114096.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("1589114096.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("1589114096.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("1589114096.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("1589114096.1234567e0"); + } +} diff --git a/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestTimestampLegacy.java b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestTimestampLegacy.java new file mode 100644 index 0000000000000..8e2a0e280e690 --- /dev/null +++ b/presto-main/src/test/java/io/prestosql/operator/scalar/timestamp/TestTimestampLegacy.java @@ -0,0 +1,23 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.operator.scalar.timestamp; + +public class TestTimestampLegacy + extends BaseTestTimestamp +{ + protected TestTimestampLegacy() + { + super(true); + } +} diff --git a/presto-main/src/test/java/io/prestosql/sql/analyzer/TestTypeSignatureTranslator.java b/presto-main/src/test/java/io/prestosql/sql/analyzer/TestTypeSignatureTranslator.java index c06ff10890362..adb517873db7a 100644 --- a/presto-main/src/test/java/io/prestosql/sql/analyzer/TestTypeSignatureTranslator.java +++ b/presto-main/src/test/java/io/prestosql/sql/analyzer/TestTypeSignatureTranslator.java @@ -50,7 +50,9 @@ public void testSimpleTypes() public void testDayTimeTypes() { assertRoundTrip("TIMESTAMP"); + assertRoundTrip("TIMESTAMP(1)"); assertRoundTrip("TIMESTAMP WITHOUT TIME ZONE"); + assertRoundTrip("TIMESTAMP(1) WITHOUT TIME ZONE"); assertRoundTrip("TIMESTAMP WITH TIME ZONE"); assertRoundTrip("TIME"); assertRoundTrip("TIME WITHOUT TIME ZONE"); diff --git a/presto-main/src/test/java/io/prestosql/sql/gen/TestExpressionCompiler.java b/presto-main/src/test/java/io/prestosql/sql/gen/TestExpressionCompiler.java index 15ec246ec139b..e28333bbaba51 100644 --- a/presto-main/src/test/java/io/prestosql/sql/gen/TestExpressionCompiler.java +++ b/presto-main/src/test/java/io/prestosql/sql/gen/TestExpressionCompiler.java @@ -33,6 +33,17 @@ import io.prestosql.operator.scalar.JsonPath; import io.prestosql.operator.scalar.MathFunctions; import io.prestosql.operator.scalar.StringFunctions; +import io.prestosql.operator.scalar.timestamp.ExtractDay; +import io.prestosql.operator.scalar.timestamp.ExtractDayOfWeek; +import io.prestosql.operator.scalar.timestamp.ExtractDayOfYear; +import io.prestosql.operator.scalar.timestamp.ExtractHour; +import io.prestosql.operator.scalar.timestamp.ExtractMinute; +import io.prestosql.operator.scalar.timestamp.ExtractMonth; +import io.prestosql.operator.scalar.timestamp.ExtractQuarter; +import io.prestosql.operator.scalar.timestamp.ExtractSecond; +import io.prestosql.operator.scalar.timestamp.ExtractWeekOfYear; +import io.prestosql.operator.scalar.timestamp.ExtractYear; +import io.prestosql.operator.scalar.timestamp.ExtractYearOfWeek; import io.prestosql.spi.connector.ConnectorSession; import io.prestosql.spi.type.ArrayType; import io.prestosql.spi.type.SqlDecimal; @@ -873,7 +884,7 @@ public void testTryCast() assertExecute("try_cast('foo' as varchar)", VARCHAR, "foo"); assertExecute("try_cast('foo' as bigint)", BIGINT, null); assertExecute("try_cast('foo' as integer)", INTEGER, null); - assertExecute("try_cast('2001-08-22' as timestamp)", TIMESTAMP, sqlTimestampOf(2001, 8, 22, 0, 0, 0, 0, TEST_SESSION)); + assertExecute("try_cast('2001-08-22' as timestamp)", TIMESTAMP, sqlTimestampOf(3, 2001, 8, 22, 0, 0, 0, 0, TEST_SESSION)); assertExecute("try_cast(bound_string as bigint)", BIGINT, null); assertExecute("try_cast(cast(null as varchar) as bigint)", BIGINT, null); assertExecute("try_cast(bound_long / 13 as bigint)", BIGINT, 94L); @@ -1519,7 +1530,7 @@ public void testExtract() Long millis = null; if (left != null) { millis = left.getMillis(); - expected = callExtractFunction(TEST_SESSION.toConnectorSession(), millis, field); + expected = callExtractFunction(TEST_SESSION.toConnectorSession(), millis, 3, field); } DateTimeZone zone = getDateTimeZone(TEST_SESSION.getTimeZoneKey()); long zoneOffsetMinutes = millis != null ? MILLISECONDS.toMinutes(zone.getOffset(millis)) : 0; @@ -1536,35 +1547,35 @@ public void testExtract() } @SuppressWarnings("fallthrough") - private static long callExtractFunction(ConnectorSession session, long value, Field field) + private static long callExtractFunction(ConnectorSession session, long value, int precision, Field field) { switch (field) { case YEAR: - return DateTimeFunctions.yearFromTimestamp(session, value); + return ExtractYear.extract(precision, session, value); case QUARTER: - return DateTimeFunctions.quarterFromTimestamp(session, value); + return ExtractQuarter.extract(precision, session, value); case MONTH: - return DateTimeFunctions.monthFromTimestamp(session, value); + return ExtractMonth.extract(precision, session, value); case WEEK: - return DateTimeFunctions.weekFromTimestamp(session, value); + return ExtractWeekOfYear.extract(precision, session, value); case DAY: case DAY_OF_MONTH: - return DateTimeFunctions.dayFromTimestamp(session, value); + return ExtractDay.extract(precision, session, value); case DAY_OF_WEEK: case DOW: - return DateTimeFunctions.dayOfWeekFromTimestamp(session, value); + return ExtractDayOfWeek.extract(precision, session, value); case YEAR_OF_WEEK: case YOW: - return DateTimeFunctions.yearOfWeekFromTimestamp(session, value); + return ExtractYearOfWeek.extract(precision, session, value); case DAY_OF_YEAR: case DOY: - return DateTimeFunctions.dayOfYearFromTimestamp(session, value); + return ExtractDayOfYear.extract(precision, session, value); case HOUR: - return DateTimeFunctions.hourFromTimestamp(session, value); + return ExtractHour.extract(precision, session, value); case MINUTE: - return DateTimeFunctions.minuteFromTimestamp(session, value); + return ExtractMinute.extract(precision, session, value); case SECOND: - return DateTimeFunctions.secondFromTimestamp(value); + return ExtractSecond.extract(precision, session, value); case TIMEZONE_MINUTE: return DateTimeFunctions.timeZoneMinuteFromTimestampWithTimeZone(packDateTimeWithZone(value, session.getTimeZoneKey())); case TIMEZONE_HOUR: diff --git a/presto-main/src/test/java/io/prestosql/sql/query/QueryAssertions.java b/presto-main/src/test/java/io/prestosql/sql/query/QueryAssertions.java index 0fa304421ca57..76aaee82380fc 100644 --- a/presto-main/src/test/java/io/prestosql/sql/query/QueryAssertions.java +++ b/presto-main/src/test/java/io/prestosql/sql/query/QueryAssertions.java @@ -16,6 +16,9 @@ import io.prestosql.Session; import io.prestosql.execution.warnings.WarningCollector; import io.prestosql.spi.PrestoException; +import io.prestosql.spi.type.SqlTime; +import io.prestosql.spi.type.SqlTimestamp; +import io.prestosql.spi.type.Type; import io.prestosql.sql.planner.Plan; import io.prestosql.sql.planner.assertions.PlanAssert; import io.prestosql.sql.planner.assertions.PlanMatchPattern; @@ -33,10 +36,12 @@ import java.io.Closeable; import java.util.List; +import java.util.Objects; import java.util.function.BiFunction; import java.util.stream.Collectors; import static io.airlift.testing.Assertions.assertEqualsIgnoreOrder; +import static io.prestosql.sql.query.QueryAssertions.ExpressionAssert.newExpressionAssert; import static io.prestosql.sql.query.QueryAssertions.QueryAssert.newQueryAssert; import static io.prestosql.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; @@ -45,7 +50,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.fail; -class QueryAssertions +public class QueryAssertions implements Closeable { private final QueryRunner runner; @@ -88,6 +93,16 @@ public AssertProvider query(@Language("SQL") String query, Session return newQueryAssert(query, runner, session); } + public AssertProvider expression(@Language("SQL") String expression) + { + return expression(expression, runner.getDefaultSession()); + } + + public AssertProvider expression(@Language("SQL") String expression, Session session) + { + return newExpressionAssert(expression, runner, session); + } + /** * @deprecated use {@link org.assertj.core.api.Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)}: *

@@ -314,4 +329,84 @@ public QueryAssert matches(@Language("SQL") String query)
             });
         }
     }
+
+    public static class ExpressionAssert
+            extends AbstractAssert
+    {
+        private static final StandardRepresentation TYPE_RENDERER = new StandardRepresentation()
+        {
+            @Override
+            public String toStringOf(Object object)
+            {
+                if (object instanceof SqlTimestamp) {
+                    SqlTimestamp timestamp = (SqlTimestamp) object;
+                    return String.format(
+                            "%s [p = %s, epochMicros = %s, fraction = %s, tz = %s]",
+                            timestamp,
+                            timestamp.getPrecision(),
+                            timestamp.getEpochMicros(),
+                            timestamp.getPicosOfMicros(),
+                            timestamp.getSessionTimeZoneKey().map(Object::toString).orElse("ø"));
+                }
+                else if (object instanceof SqlTime) {
+                    SqlTime time = (SqlTime) object;
+                    return String.format(
+                            "%s [millis = %s, tz = %s]",
+                            time,
+                            time.getMillis(),
+                            time.getSessionTimeZoneKey().map(Object::toString).orElse("ø"));
+                }
+
+                return Objects.toString(object);
+            }
+        };
+
+        private final QueryRunner runner;
+        private final Session session;
+        private final Type actualType;
+
+        static AssertProvider newExpressionAssert(String expression, QueryRunner runner, Session session)
+        {
+            MaterializedResult result = runner.execute(session, "VALUES " + expression);
+            Type type = result.getTypes().get(0);
+            Object value = result.getOnlyColumnAsSet().iterator().next();
+            return () -> new ExpressionAssert(runner, session, value, type)
+                    .withRepresentation(TYPE_RENDERER);
+        }
+
+        public ExpressionAssert(QueryRunner runner, Session session, Object actual, Type actualType)
+        {
+            super(actual, Object.class);
+            this.runner = runner;
+            this.session = session;
+            this.actualType = actualType;
+        }
+
+        public ExpressionAssert isEqualTo(BiFunction evaluator)
+        {
+            return isEqualTo(evaluator.apply(session, runner));
+        }
+
+        public ExpressionAssert matches(@Language("SQL") String expression)
+        {
+            MaterializedResult result = runner.execute(session, "VALUES " + expression);
+            Type expectedType = result.getTypes().get(0);
+            Object expectedValue = result.getOnlyColumnAsSet().iterator().next();
+
+            return satisfies(actual -> {
+                assertThat(actualType).as("Type")
+                        .isEqualTo(expectedType);
+
+                assertThat(actual)
+                        .withRepresentation(TYPE_RENDERER)
+                        .isEqualTo(expectedValue);
+            });
+        }
+
+        public ExpressionAssert hasType(Type type)
+        {
+            objects.assertEqual(info, actualType, type);
+            return this;
+        }
+    }
 }
diff --git a/presto-main/src/test/java/io/prestosql/type/TestArrayOperators.java b/presto-main/src/test/java/io/prestosql/type/TestArrayOperators.java
index 5fbb7e509cc73..a915950b54db2 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestArrayOperators.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestArrayOperators.java
@@ -64,7 +64,7 @@
 import static io.prestosql.spi.type.IntegerType.INTEGER;
 import static io.prestosql.spi.type.RealType.REAL;
 import static io.prestosql.spi.type.SmallintType.SMALLINT;
-import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
+import static io.prestosql.spi.type.TimestampType.createTimestampType;
 import static io.prestosql.spi.type.TinyintType.TINYINT;
 import static io.prestosql.spi.type.VarcharType.VARCHAR;
 import static io.prestosql.spi.type.VarcharType.createVarcharType;
@@ -206,7 +206,7 @@ public void testArrayToJson()
         assertFunction(
                 "CAST(ARRAY[TIMESTAMP '1970-01-01 00:00:01', null] AS JSON)",
                 JSON,
-                format("[\"%s\",null]", sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
+                format("[\"%s\",null]", sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
         assertFunction(
                 "CAST(ARRAY[DATE '2001-08-22', DATE '2001-08-23', null] AS JSON)",
                 JSON,
@@ -368,10 +368,10 @@ public void testConstructor()
         assertFunction("ARRAY [TRUE, FALSE]", new ArrayType(BOOLEAN), ImmutableList.of(true, false));
         assertFunction(
                 "ARRAY [TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01']",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
-                        sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
         assertFunction("ARRAY [sqrt(-1)]", new ArrayType(DOUBLE), ImmutableList.of(NaN));
         assertFunction("ARRAY [pow(infinity(), 2)]", new ArrayType(DOUBLE), ImmutableList.of(POSITIVE_INFINITY));
         assertFunction("ARRAY [pow(-infinity(), 1)]", new ArrayType(DOUBLE), ImmutableList.of(NEGATIVE_INFINITY));
@@ -405,9 +405,9 @@ public void testArrayToArrayConcat()
         assertFunction("ARRAY ['puppies'] || ARRAY ['kittens']", new ArrayType(createVarcharType(7)), ImmutableList.of("puppies", "kittens"));
         assertFunction("ARRAY [TRUE] || ARRAY [FALSE]", new ArrayType(BOOLEAN), ImmutableList.of(true, false));
         assertFunction("concat(ARRAY [1] , ARRAY[2,3])", new ArrayType(INTEGER), ImmutableList.of(1, 2, 3));
-        assertFunction("ARRAY [TIMESTAMP '1970-01-01 00:00:01'] || ARRAY[TIMESTAMP '1973-07-08 22:00:01']", new ArrayType(TIMESTAMP), ImmutableList.of(
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
-                sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
+        assertFunction("ARRAY [TIMESTAMP '1970-01-01 00:00:01'] || ARRAY[TIMESTAMP '1973-07-08 22:00:01']", new ArrayType(createTimestampType(0)), ImmutableList.of(
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
+                sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
         assertFunction("ARRAY [ARRAY[ARRAY[1]]] || ARRAY [ARRAY[ARRAY[2]]]",
                 new ArrayType(new ArrayType(new ArrayType(INTEGER))),
                 asList(singletonList(Ints.asList(1)), singletonList(Ints.asList(2))));
@@ -467,12 +467,12 @@ public void testElementArrayConcat()
         assertFunction("ARRAY [2.0E0] || 1.0E0", new ArrayType(DOUBLE), Lists.newArrayList(2.0, 1.0));
         assertFunction("'puppies' || ARRAY ['kittens']", new ArrayType(createVarcharType(7)), Lists.newArrayList("puppies", "kittens"));
         assertFunction("ARRAY ['kittens'] || 'puppies'", new ArrayType(createVarcharType(7)), Lists.newArrayList("kittens", "puppies"));
-        assertFunction("ARRAY [TIMESTAMP '1970-01-01 00:00:01'] || TIMESTAMP '1973-07-08 22:00:01'", new ArrayType(TIMESTAMP), ImmutableList.of(
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
-                sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
-        assertFunction("TIMESTAMP '1973-07-08 22:00:01' || ARRAY [TIMESTAMP '1970-01-01 00:00:01']", new ArrayType(TIMESTAMP), ImmutableList.of(
-                sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
+        assertFunction("ARRAY [TIMESTAMP '1970-01-01 00:00:01'] || TIMESTAMP '1973-07-08 22:00:01'", new ArrayType(createTimestampType(0)), ImmutableList.of(
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
+                sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
+        assertFunction("TIMESTAMP '1973-07-08 22:00:01' || ARRAY [TIMESTAMP '1970-01-01 00:00:01']", new ArrayType(createTimestampType(0)), ImmutableList.of(
+                sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
         assertFunction("ARRAY [2, 8] || ARRAY[ARRAY[3, 6], ARRAY[4]]", new ArrayType(new ArrayType(INTEGER)), ImmutableList.of(ImmutableList.of(2, 8), ImmutableList.of(3, 6), ImmutableList.of(4)));
         assertFunction("ARRAY [ARRAY [1], ARRAY [2, 8]] || ARRAY [3, 6]", new ArrayType(new ArrayType(INTEGER)), ImmutableList.of(ImmutableList.of(1), ImmutableList.of(2, 8), ImmutableList.of(3, 6)));
         assertFunction(
@@ -552,16 +552,16 @@ public void testArrayJoin()
         assertFunction("ARRAY_JOIN(ARRAY [sqrt(-1), infinity()], ',')", VARCHAR, "NaN,Infinity");
         assertFunction("ARRAY_JOIN(ARRAY [TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'], '|')", VARCHAR, format(
                 "%s|%s",
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
-                sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
+                sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
         assertFunction(
                 "ARRAY_JOIN(ARRAY [null, TIMESTAMP '1970-01-01 00:00:01'], '|')",
                 VARCHAR,
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION).toString());
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION).toString());
         assertFunction(
                 "ARRAY_JOIN(ARRAY [null, TIMESTAMP '1970-01-01 00:00:01'], '|', 'XYZ')",
                 VARCHAR,
-                "XYZ|" + sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION).toString());
+                "XYZ|" + sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION).toString());
         assertFunction("ARRAY_JOIN(ARRAY [1.0, 2.1, 3.3], 'x')", VARCHAR, "1.0x2.1x3.3");
         assertFunction("ARRAY_JOIN(ARRAY [1.0, 2.100, 3.3], 'x')", VARCHAR, "1.000x2.100x3.300");
         assertFunction("ARRAY_JOIN(ARRAY [1.0, 2.100, NULL], 'x', 'N/A')", VARCHAR, "1.000x2.100xN/A");
@@ -705,8 +705,8 @@ public void testSubscript()
         assertFunction("ARRAY [TRUE, FALSE][2]", BOOLEAN, false);
         assertFunction(
                 "ARRAY [TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'][1]",
-                TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
+                createTimestampType(0),
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
         assertFunction("ARRAY [infinity()][1]", DOUBLE, POSITIVE_INFINITY);
         assertFunction("ARRAY [-infinity()][1]", DOUBLE, NEGATIVE_INFINITY);
         assertFunction("ARRAY [sqrt(-1)][1]", DOUBLE, NaN);
@@ -757,12 +757,12 @@ public void testElementAt()
         assertFunction("ELEMENT_AT(ARRAY [TRUE, FALSE], -1)", BOOLEAN, false);
         assertFunction(
                 "ELEMENT_AT(ARRAY [TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'], 1)",
-                TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
+                createTimestampType(0),
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
         assertFunction(
                 "ELEMENT_AT(ARRAY [TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'], -2)",
-                TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
+                createTimestampType(0),
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
         assertFunction("ELEMENT_AT(ARRAY [infinity()], 1)", DOUBLE, POSITIVE_INFINITY);
         assertFunction("ELEMENT_AT(ARRAY [infinity()], -1)", DOUBLE, POSITIVE_INFINITY);
         assertFunction("ELEMENT_AT(ARRAY [-infinity()], 1)", DOUBLE, NEGATIVE_INFINITY);
@@ -801,11 +801,11 @@ public void testSort()
         assertFunction("ARRAY_SORT(ARRAY[22.1E0, 11.1E0, 1.1E0, 44.1E0])", new ArrayType(DOUBLE), ImmutableList.of(1.1, 11.1, 22.1, 44.1));
         assertFunction(
                 "ARRAY_SORT(ARRAY [TIMESTAMP '1973-07-08 22:00:01', TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1989-02-06 12:00:00'])",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
-                        sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
-                        sqlTimestampOf(1989, 2, 6, 12, 0, 0, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1989, 2, 6, 12, 0, 0, 0, TEST_SESSION)));
         assertFunction("ARRAY_SORT(ARRAY [ARRAY [1], ARRAY [2]])",
                 new ArrayType(new ArrayType(INTEGER)),
                 ImmutableList.of(ImmutableList.of(1), ImmutableList.of(2)));
@@ -882,13 +882,13 @@ public void testSort()
                         "WHEN date_diff('millisecond', y, x) < 0 THEN 1 " +
                         "WHEN date_diff('millisecond', y, x) = 0 THEN 0 " +
                         "ELSE -1 END)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 asList(
                         null,
                         null,
-                        sqlTimestampOf(1989, 2, 6, 12, 0, 0, 0, TEST_SESSION),
-                        sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
-                        sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 1989, 2, 6, 12, 0, 0, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
         assertFunction(
                 "ARRAY_SORT(ARRAY[ARRAY[2, 3, 1], null, ARRAY[4, null, 2, 1, 4], ARRAY[1, 2], null], (x, y) -> CASE " +
                         "WHEN x IS NULL THEN -1 " +
@@ -965,10 +965,10 @@ public void testDistinct()
         assertFunction("ARRAY_DISTINCT(ARRAY [TRUE, FALSE, FALSE, TRUE])", new ArrayType(BOOLEAN), ImmutableList.of(true, false));
         assertFunction(
                 "ARRAY_DISTINCT(ARRAY [TIMESTAMP '1973-07-08 22:00:01', TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'])",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
-                        sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
         assertFunction("ARRAY_DISTINCT(ARRAY ['2', '3', '2'])", new ArrayType(createVarcharType(1)), ImmutableList.of("2", "3"));
         assertFunction("ARRAY_DISTINCT(ARRAY ['BB', 'CCC', 'BB'])", new ArrayType(createVarcharType(3)), ImmutableList.of("BB", "CCC"));
         assertFunction(
@@ -1639,48 +1639,48 @@ public void testSequenceDateTimeDayToSecond()
 
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:00:10', timestamp '2016-04-16 01:07:00', interval '3' minute)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 3, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 6, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 3, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 6, 10, 0, TEST_SESSION)));
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:10:10', timestamp '2016-04-16 01:03:00', interval '-3' minute)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 10, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 7, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 4, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 10, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 7, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 4, 10, 0, TEST_SESSION)));
 
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:00:10', timestamp '2016-04-16 01:01:00', interval '20' second)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 30, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 50, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 30, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 50, 0, TEST_SESSION)));
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:01:10', timestamp '2016-04-16 01:00:20', interval '-20' second)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 1, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 50, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 30, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 1, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 50, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 30, 0, TEST_SESSION)));
 
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:00:10', timestamp '2016-04-18 01:01:00', interval '19' hour)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 16, 20, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 17, 15, 0, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 16, 20, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 17, 15, 0, 10, 0, TEST_SESSION)));
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:00:10', timestamp '2016-04-14 01:00:20', interval '-19' hour)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 15, 6, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 4, 14, 11, 0, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 15, 6, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 4, 14, 11, 0, 10, 0, TEST_SESSION)));
 
         // failure modes
         assertInvalidFunction(
@@ -1746,33 +1746,33 @@ public void testSequenceDateTimeYearToMonth()
 
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:00:10', timestamp '2016-09-16 01:10:00', interval '2' month)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 6, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 8, 16, 1, 0, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 6, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 8, 16, 1, 0, 10, 0, TEST_SESSION)));
         assertFunction(
                 "SEQUENCE(timestamp '2016-09-16 01:10:10', timestamp '2016-04-16 01:00:00', interval '-2' month)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 9, 16, 1, 10, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 7, 16, 1, 10, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2016, 5, 16, 1, 10, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 9, 16, 1, 10, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 7, 16, 1, 10, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2016, 5, 16, 1, 10, 10, 0, TEST_SESSION)));
 
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:00:10', timestamp '2021-04-16 01:01:00', interval '2' year)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2018, 4, 16, 1, 0, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2020, 4, 16, 1, 0, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2018, 4, 16, 1, 0, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2020, 4, 16, 1, 0, 10, 0, TEST_SESSION)));
         assertFunction(
                 "SEQUENCE(timestamp '2016-04-16 01:01:10', timestamp '2011-04-16 01:00:00', interval '-2' year)",
-                new ArrayType(TIMESTAMP),
+                new ArrayType(createTimestampType(0)),
                 ImmutableList.of(
-                        sqlTimestampOf(2016, 4, 16, 1, 1, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2014, 4, 16, 1, 1, 10, 0, TEST_SESSION),
-                        sqlTimestampOf(2012, 4, 16, 1, 1, 10, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 2016, 4, 16, 1, 1, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2014, 4, 16, 1, 1, 10, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 2012, 4, 16, 1, 1, 10, 0, TEST_SESSION)));
 
         // failure modes
         assertInvalidFunction(
diff --git a/presto-main/src/test/java/io/prestosql/type/TestDateBase.java b/presto-main/src/test/java/io/prestosql/type/TestDateBase.java
index 14d3cf4639e06..678add188a62d 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestDateBase.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestDateBase.java
@@ -129,7 +129,7 @@ public void testCastToTimestamp()
     {
         assertFunction("cast(DATE '2001-1-22' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 0, 0, 0, 0, session));
+                sqlTimestampOf(3, 2001, 1, 22, 0, 0, 0, 0, session));
     }
 
     @Test
diff --git a/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperators.java b/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperators.java
index bca34c9717986..c4d8099fe7ad0 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperators.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperators.java
@@ -31,7 +31,7 @@
 import static io.prestosql.spi.type.TimeType.TIME;
 import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
 import static io.prestosql.spi.type.TimeZoneKey.getTimeZoneKey;
-import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
+import static io.prestosql.spi.type.TimestampType.createTimestampType;
 import static io.prestosql.testing.DateTimeTestingUtils.sqlTimestampOf;
 import static io.prestosql.testing.TestingSession.testSessionBuilder;
 
@@ -50,29 +50,29 @@ public void testTimeZoneGap()
 
         assertFunction(
                 "TIMESTAMP '2013-03-31 00:05' + INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-03-31 00:05' + INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 2, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 2, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-03-31 00:05' + INTERVAL '3' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 3, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 3, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
 
         assertFunction(
                 "TIMESTAMP '2013-03-31 04:05' - INTERVAL '3' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-03-31 03:05' - INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-03-31 01:05' - INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 0, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 0, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
     }
 
     @Test
@@ -80,43 +80,43 @@ public void testDaylightTimeSaving()
     {
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 2, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 2, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
 
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '3' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 3, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 3, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '4' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 4, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 4, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
 
         assertFunction(
                 "TIMESTAMP '2013-10-27 03:05' - INTERVAL '4' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 26, 23, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 26, 23, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-10-27 02:05' - INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-10-27 01:05' - INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
 
         assertFunction(
                 "TIMESTAMP '2013-10-27 03:05' - INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 2, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 2, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
         assertFunction(
                 "TIMESTAMP '2013-10-27 03:05' - INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 1, 5, 0, 0, DATE_TIME_ZONE, TIME_ZONE_KEY, session.toConnectorSession()));
     }
 
     @Test
diff --git a/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsBase.java b/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsBase.java
index de5e2422c1b39..eb35936ae2fc0 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsBase.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsBase.java
@@ -123,28 +123,28 @@ public void testTimestampPlusInterval()
     {
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321' + INTERVAL '3' hour",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 6, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 6, 4, 5, 321, session));
         assertFunction("INTERVAL '3' hour + TIMESTAMP '2001-1-22 03:04:05.321'",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 6, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 6, 4, 5, 321, session));
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321' + INTERVAL '3' day",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 25, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 25, 3, 4, 5, 321, session));
         assertFunction("INTERVAL '3' day + TIMESTAMP '2001-1-22 03:04:05.321'",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 25, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 25, 3, 4, 5, 321, session));
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321' + INTERVAL '3' month",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 4, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 4, 22, 3, 4, 5, 321, session));
         assertFunction("INTERVAL '3' month + TIMESTAMP '2001-1-22 03:04:05.321'",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 4, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 4, 22, 3, 4, 5, 321, session));
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321' + INTERVAL '3' year",
                 TIMESTAMP,
-                sqlTimestampOf(2004, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2004, 1, 22, 3, 4, 5, 321, session));
         assertFunction("INTERVAL '3' year + TIMESTAMP '2001-1-22 03:04:05.321'",
                 TIMESTAMP,
-                sqlTimestampOf(2004, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2004, 1, 22, 3, 4, 5, 321, session));
 
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321 +05:09' + INTERVAL '3' hour",
                 TIMESTAMP_WITH_TIME_ZONE,
@@ -212,13 +212,13 @@ public void testTimestampMinusInterval()
     {
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321' - INTERVAL '3' day",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 19, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 19, 3, 4, 5, 321, session));
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321 +05:09' - INTERVAL '3' day",
                 TIMESTAMP_WITH_TIME_ZONE,
                 new SqlTimestampWithTimeZone(new DateTime(2001, 1, 19, 3, 4, 5, 321, WEIRD_TIME_ZONE).getMillis(), WEIRD_TIME_ZONE_KEY));
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321' - INTERVAL '3' month",
                 TIMESTAMP,
-                sqlTimestampOf(2000, 10, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2000, 10, 22, 3, 4, 5, 321, session));
         assertFunction("TIMESTAMP '2001-1-22 03:04:05.321 +05:09' - INTERVAL '3' month",
                 TIMESTAMP_WITH_TIME_ZONE,
                 new SqlTimestampWithTimeZone(new DateTime(2000, 10, 22, 3, 4, 5, 321, WEIRD_TIME_ZONE).getMillis(), WEIRD_TIME_ZONE_KEY));
diff --git a/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsLegacy.java b/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsLegacy.java
index 34811030dd024..0625b5565aef2 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsLegacy.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestDateTimeOperatorsLegacy.java
@@ -31,7 +31,7 @@
 import static io.prestosql.spi.type.TimeType.TIME;
 import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
 import static io.prestosql.spi.type.TimeZoneKey.getTimeZoneKey;
-import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
+import static io.prestosql.spi.type.TimestampType.createTimestampType;
 import static io.prestosql.testing.DateTimeTestingUtils.sqlTimestampOf;
 import static io.prestosql.testing.TestingSession.testSessionBuilder;
 import static java.util.concurrent.TimeUnit.HOURS;
@@ -51,29 +51,29 @@ public void testTimeZoneGap()
 
         assertFunction(
                 "TIMESTAMP '2013-03-31 00:05' + INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 1, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 1, 5, 0, 0, session));
         assertFunction(
                 "TIMESTAMP '2013-03-31 00:05' + INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 3, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 3, 5, 0, 0, session));
         assertFunction(
                 "TIMESTAMP '2013-03-31 00:05' + INTERVAL '3' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 4, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 4, 5, 0, 0, session));
 
         assertFunction(
                 "TIMESTAMP '2013-03-31 04:05' - INTERVAL '3' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 0, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 0, 5, 0, 0, session));
         assertFunction(
                 "TIMESTAMP '2013-03-31 03:05' - INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 0, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 0, 5, 0, 0, session));
         assertFunction(
                 "TIMESTAMP '2013-03-31 01:05' - INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 31, 0, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 3, 31, 0, 5, 0, 0, session));
     }
 
     @Test
@@ -82,43 +82,43 @@ public void testDaylightTimeSaving()
         // See testDaylightTimeSavingSwitchCrossingIsNotApplied for new semantics
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 1, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 1, 5, 0, 0, session));
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 2, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 2, 5, 0, 0, session));
         // we need to manipulate millis directly here because 2 am has two representations in out time zone, and we need the second one
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '3' hour",
-                TIMESTAMP,
-                sqlTimestampOf(new DateTime(2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE).plus(HOURS.toMillis(3)), session));
+                createTimestampType(3),
+                sqlTimestampOf(3, new DateTime(2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE).plus(HOURS.toMillis(3)), session));
         assertFunction(
                 "TIMESTAMP '2013-10-27 00:05' + INTERVAL '4' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 3, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 3, 5, 0, 0, session));
 
         assertFunction(
                 "TIMESTAMP '2013-10-27 03:05' - INTERVAL '4' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 0, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 0, 5, 0, 0, session));
         assertFunction(
                 "TIMESTAMP '2013-10-27 02:05' - INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 0, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 0, 5, 0, 0, session));
         assertFunction(
                 "TIMESTAMP '2013-10-27 01:05' - INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 0, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 0, 5, 0, 0, session));
 
         assertFunction(
                 "TIMESTAMP '2013-10-27 03:05' - INTERVAL '1' hour",
-                TIMESTAMP,
-                sqlTimestampOf(new DateTime(2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE).plus(HOURS.toMillis(3)), session));
+                createTimestampType(3),
+                sqlTimestampOf(3, new DateTime(2013, 10, 27, 0, 5, 0, 0, DATE_TIME_ZONE).plus(HOURS.toMillis(3)), session));
         assertFunction(
                 "TIMESTAMP '2013-10-27 03:05' - INTERVAL '2' hour",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 10, 27, 2, 5, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2013, 10, 27, 2, 5, 0, 0, session));
     }
 
     @Test
diff --git a/presto-main/src/test/java/io/prestosql/type/TestJsonOperators.java b/presto-main/src/test/java/io/prestosql/type/TestJsonOperators.java
index 712550a4d71b7..9c06e2f321ec9 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestJsonOperators.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestJsonOperators.java
@@ -393,7 +393,7 @@ public void testCastFromVarchar()
     public void testCastFromTimestamp()
     {
         assertFunction("cast(cast (null as timestamp) as JSON)", JSON, null);
-        assertFunction("CAST(TIMESTAMP '1970-01-01 00:00:01' AS JSON)", JSON, format("\"%s\"", sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
+        assertFunction("CAST(TIMESTAMP '1970-01-01 00:00:01' AS JSON)", JSON, format("\"%s\"", sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
     }
 
     @Test
diff --git a/presto-main/src/test/java/io/prestosql/type/TestMapOperators.java b/presto-main/src/test/java/io/prestosql/type/TestMapOperators.java
index 2739d0523d153..ae75efd451e4e 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestMapOperators.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestMapOperators.java
@@ -48,7 +48,7 @@
 import static io.prestosql.spi.type.IntegerType.INTEGER;
 import static io.prestosql.spi.type.RealType.REAL;
 import static io.prestosql.spi.type.SmallintType.SMALLINT;
-import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
+import static io.prestosql.spi.type.TimestampType.createTimestampType;
 import static io.prestosql.spi.type.TinyintType.TINYINT;
 import static io.prestosql.spi.type.VarbinaryType.VARBINARY;
 import static io.prestosql.spi.type.VarcharType.VARCHAR;
@@ -101,19 +101,19 @@ public void testConstructor()
         assertFunction("MAP(ARRAY[TRUE, FALSE], ARRAY[2,4])", mapType(BOOLEAN, INTEGER), ImmutableMap.of(true, 2, false, 4));
         assertFunction(
                 "MAP(ARRAY['1', '100'], ARRAY[TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'])",
-                mapType(createVarcharType(3), TIMESTAMP),
+                mapType(createVarcharType(3), createTimestampType(0)),
                 ImmutableMap.of(
                         "1",
-                        sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
                         "100",
-                        sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
+                        sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION)));
         assertFunction(
                 "MAP(ARRAY[TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'], ARRAY[1.0E0, 100.0E0])",
-                mapType(TIMESTAMP, DOUBLE),
+                mapType(createTimestampType(0), DOUBLE),
                 ImmutableMap.of(
-                        sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION),
                         1.0,
-                        sqlTimestampOf(1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
+                        sqlTimestampOf(0, 1973, 7, 8, 22, 0, 1, 0, TEST_SESSION),
                         100.0));
 
         assertInvalidFunction("MAP(ARRAY [1], ARRAY [2, 4])", "Key and value arrays must be the same length");
@@ -246,7 +246,7 @@ public void testMapToJson()
         assertFunction(
                 "CAST(MAP(ARRAY[1, 2], ARRAY[TIMESTAMP '1970-01-01 00:00:01', null]) AS JSON)",
                 JSON,
-                format("{\"1\":\"%s\",\"2\":null}", sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION).toString()));
+                format("{\"1\":\"%s\",\"2\":null}", sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION).toString()));
         assertFunction(
                 "CAST(MAP(ARRAY[2, 5, 3], ARRAY[DATE '2001-08-22', DATE '2001-08-23', null]) AS JSON)",
                 JSON,
@@ -517,8 +517,8 @@ public void testElementAt()
         assertFunction("element_at(MAP(ARRAY [ARRAY [1, 2], ARRAY [3]], ARRAY [1e0, 2e0]), ARRAY [1, 2])", DOUBLE, 1.0);
         assertFunction(
                 "element_at(MAP(ARRAY ['1', '100'], ARRAY [TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '2005-09-10 13:00:00']), '1')",
-                TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
+                createTimestampType(0),
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
         assertFunction("element_at(MAP(ARRAY [from_unixtime(1), from_unixtime(100)], ARRAY [1.0E0, 100.0E0]), from_unixtime(1))", DOUBLE, 1.0);
     }
 
@@ -542,8 +542,8 @@ public void testSubscript()
         assertFunction("MAP(ARRAY[TRUE,FALSE],ARRAY[2,4])[TRUE]", INTEGER, 2);
         assertFunction(
                 "MAP(ARRAY['1', '100'], ARRAY[TIMESTAMP '1970-01-01 00:00:01', TIMESTAMP '1973-07-08 22:00:01'])['1']",
-                TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
+                createTimestampType(0),
+                sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION));
         assertFunction("MAP(ARRAY[from_unixtime(1), from_unixtime(100)], ARRAY[1.0E0, 100.0E0])[from_unixtime(1)]", DOUBLE, 1.0);
         assertInvalidFunction("MAP(ARRAY [BIGINT '1'], ARRAY [BIGINT '2'])[3]", "Key not present in map: 3");
         assertInvalidFunction("MAP(ARRAY ['hi'], ARRAY [2])['missing']", "Key not present in map: missing");
@@ -562,8 +562,8 @@ public void testMapKeys()
         assertFunction("MAP_KEYS(MAP(ARRAY[TRUE], ARRAY[2]))", new ArrayType(BOOLEAN), ImmutableList.of(true));
         assertFunction(
                 "MAP_KEYS(MAP(ARRAY[TIMESTAMP '1970-01-01 00:00:01'], ARRAY[1.0E0]))",
-                new ArrayType(TIMESTAMP),
-                ImmutableList.of(sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
+                new ArrayType(createTimestampType(0)),
+                ImmutableList.of(sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
         assertFunction("MAP_KEYS(MAP(ARRAY[CAST('puppies' as varbinary)], ARRAY['kittens']))", new ArrayType(VARBINARY), ImmutableList.of(sqlVarbinary("puppies")));
         assertFunction("MAP_KEYS(MAP(ARRAY[1,2],  ARRAY[ARRAY[1, 2], ARRAY[3]]))", new ArrayType(INTEGER), ImmutableList.of(1, 2));
         assertFunction("MAP_KEYS(MAP(ARRAY[1,4], ARRAY[MAP(ARRAY[2], ARRAY[3]), MAP(ARRAY[5], ARRAY[6])]))", new ArrayType(INTEGER), ImmutableList.of(1, 4));
@@ -822,7 +822,7 @@ public void testMapToMapCast()
         assertFunction("CAST(MAP(ARRAY[1,2], ARRAY[array[1],array[2]]) AS MAP>)", mapType(BIGINT, new ArrayType(BOOLEAN)), ImmutableMap.of(1L, ImmutableList.of(true), 2L, ImmutableList.of(true)));
         assertFunction("CAST(MAP(ARRAY[1], ARRAY[MAP(ARRAY[1.0E0], ARRAY[false])]) AS MAP)", mapType(VARCHAR, mapType(BIGINT, BIGINT)), ImmutableMap.of("1", ImmutableMap.of(1L, 0L)));
         assertFunction("CAST(MAP(ARRAY[1,2], ARRAY[DATE '2016-01-02', DATE '2016-02-03']) AS MAP(bigint, varchar))", mapType(BIGINT, VARCHAR), ImmutableMap.of(1L, "2016-01-02", 2L, "2016-02-03"));
-        assertFunction("CAST(MAP(ARRAY[1,2], ARRAY[TIMESTAMP '2016-01-02 01:02:03', TIMESTAMP '2016-02-03 03:04:05']) AS MAP(bigint, varchar))", mapType(BIGINT, VARCHAR), ImmutableMap.of(1L, "2016-01-02 01:02:03.000", 2L, "2016-02-03 03:04:05.000"));
+        assertFunction("CAST(MAP(ARRAY[1,2], ARRAY[TIMESTAMP '2016-01-02 01:02:03', TIMESTAMP '2016-02-03 03:04:05']) AS MAP(bigint, varchar))", mapType(BIGINT, VARCHAR), ImmutableMap.of(1L, "2016-01-02 01:02:03", 2L, "2016-02-03 03:04:05"));
         assertFunction("CAST(MAP(ARRAY['123', '456'], ARRAY[1.23456E0, 2.34567E0]) AS MAP(integer, real))", mapType(INTEGER, REAL), ImmutableMap.of(123, 1.23456F, 456, 2.34567F));
         assertFunction("CAST(MAP(ARRAY['123', '456'], ARRAY[1.23456E0, 2.34567E0]) AS MAP(smallint, decimal(6,5)))", mapType(SMALLINT, createDecimalType(6, 5)), ImmutableMap.of((short) 123, SqlDecimal.of("1.23456"), (short) 456, SqlDecimal.of("2.34567")));
         assertFunction("CAST(MAP(ARRAY[json '1'], ARRAY[1]) AS MAP(bigint, bigint))", mapType(BIGINT, BIGINT), ImmutableMap.of(1L, 1L));
diff --git a/presto-main/src/test/java/io/prestosql/type/TestRowOperators.java b/presto-main/src/test/java/io/prestosql/type/TestRowOperators.java
index 25307c6a9b397..e54a664adedba 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestRowOperators.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestRowOperators.java
@@ -120,7 +120,7 @@ public void testRowToJson()
         assertFunction(
                 "CAST(ROW(TIMESTAMP '1970-01-01 00:00:01', cast(null as TIMESTAMP)) AS JSON)",
                 JSON,
-                format("[\"%s\",null]", sqlTimestampOf(1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
+                format("[\"%s\",null]", sqlTimestampOf(0, 1970, 1, 1, 0, 0, 1, 0, TEST_SESSION)));
 
         assertFunction(
                 "cast(ROW(ARRAY[1, 2], ARRAY[3, null], ARRAY[], ARRAY[null, null], CAST(null AS ARRAY(BIGINT))) AS JSON)",
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimeBase.java b/presto-main/src/test/java/io/prestosql/type/TestTimeBase.java
index 80d7d5552a7fd..2b07ef92d4e90 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimeBase.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimeBase.java
@@ -180,7 +180,7 @@ public void testCastToTimestamp()
     {
         assertFunction("cast(TIME '03:04:05.321' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 1970, 1, 1, 3, 4, 5, 321, session));
     }
 
     @Test
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZone.java b/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZone.java
index aadb387b01db1..e0ac710e62031 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZone.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZone.java
@@ -43,6 +43,6 @@ public void testCastToTimestamp()
     {
         assertFunction("cast(TIME '03:04:05.321 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 1970, 1, 1, 3, 4, 5, 321, session));
     }
 }
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZoneLegacy.java b/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZoneLegacy.java
index 47f24c9170b13..b111c378694ae 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZoneLegacy.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimeWithTimeZoneLegacy.java
@@ -43,6 +43,6 @@ public void testCastToTimestamp()
     {
         assertFunction("cast(TIME '03:04:05.321 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(1970, 1, 1, 2 /* not 3 */, 4, 5, 321, session));
+                sqlTimestampOf(3, 1970, 1, 1, 2 /* not 3 */, 4, 5, 321, session));
     }
 }
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimestamp.java b/presto-main/src/test/java/io/prestosql/type/TestTimestamp.java
index d2bf57257c16b..6653847ce2e32 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimestamp.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimestamp.java
@@ -34,35 +34,35 @@ public void testCastFromVarcharContainingTimeZone()
         assertFunction(
                 "cast('2001-1-22 03:04:05.321 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 3, 4, 5, 321_000_000)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 3, 4, 5, 321_000_000)));
         assertFunction(
                 "cast('2001-1-22 03:04:05 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 3, 4, 5)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 3, 4, 5)));
         assertFunction(
                 "cast('2001-1-22 03:04 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 3, 4, 0)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 3, 4, 0)));
         assertFunction(
                 "cast('2001-1-22 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 0, 0, 0)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 0, 0, 0)));
 
         assertFunction(
                 "cast('2001-1-22 03:04:05.321 Asia/Oral' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 3, 4, 5, 321_000_000)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 3, 4, 5, 321_000_000)));
         assertFunction(
                 "cast('2001-1-22 03:04:05 Asia/Oral' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 3, 4, 5)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 3, 4, 5)));
         assertFunction(
                 "cast('2001-1-22 03:04 Asia/Oral' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 3, 4, 0)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 3, 4, 0)));
         assertFunction(
                 "cast('2001-1-22 Asia/Oral' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(LocalDateTime.of(2001, 1, 22, 0, 0, 0)));
+                sqlTimestampOf(3, LocalDateTime.of(2001, 1, 22, 0, 0, 0)));
     }
 }
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimestampBase.java b/presto-main/src/test/java/io/prestosql/type/TestTimestampBase.java
index 134989212bdb8..04da94fd0060c 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimestampBase.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimestampBase.java
@@ -32,6 +32,7 @@
 import static io.prestosql.spi.type.TimeType.TIME;
 import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
 import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
+import static io.prestosql.spi.type.TimestampType.createTimestampType;
 import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
 import static io.prestosql.spi.type.VarcharType.VARCHAR;
 import static io.prestosql.testing.DateTimeTestingUtils.sqlTimeOf;
@@ -71,19 +72,19 @@ public void testSubtract()
     @Test
     public void testLiteral()
     {
-        assertFunction("TIMESTAMP '2013-03-30 01:05'", TIMESTAMP, sqlTimestampOf(2013, 3, 30, 1, 5, 0, 0, session));
-        assertFunction("TIMESTAMP '2013-03-30 02:05'", TIMESTAMP, sqlTimestampOf(2013, 3, 30, 2, 5, 0, 0, session));
-        assertFunction("TIMESTAMP '2013-03-30 03:05'", TIMESTAMP, sqlTimestampOf(2013, 3, 30, 3, 5, 0, 0, session));
+        assertFunction("TIMESTAMP '2013-03-30 01:05'", createTimestampType(0), sqlTimestampOf(0, 2013, 3, 30, 1, 5, 0, 0, session));
+        assertFunction("TIMESTAMP '2013-03-30 02:05'", createTimestampType(0), sqlTimestampOf(0, 2013, 3, 30, 2, 5, 0, 0, session));
+        assertFunction("TIMESTAMP '2013-03-30 03:05'", createTimestampType(0), sqlTimestampOf(0, 2013, 3, 30, 3, 5, 0, 0, session));
 
-        assertFunction("TIMESTAMP '2001-01-22 03:04:05.321'", TIMESTAMP, sqlTimestampOf(2001, 1, 22, 3, 4, 5, 321, session));
-        assertFunction("TIMESTAMP '2001-01-22 03:04:05'", TIMESTAMP, sqlTimestampOf(2001, 1, 22, 3, 4, 5, 0, session));
-        assertFunction("TIMESTAMP '2001-01-22 03:04'", TIMESTAMP, sqlTimestampOf(2001, 1, 22, 3, 4, 0, 0, session));
-        assertFunction("TIMESTAMP '2001-01-22'", TIMESTAMP, sqlTimestampOf(2001, 1, 22, 0, 0, 0, 0, session));
+        assertFunction("TIMESTAMP '2001-01-22 03:04:05.321'", createTimestampType(3), sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 321, session));
+        assertFunction("TIMESTAMP '2001-01-22 03:04:05'", createTimestampType(0), sqlTimestampOf(0, 2001, 1, 22, 3, 4, 5, 0, session));
+        assertFunction("TIMESTAMP '2001-01-22 03:04'", createTimestampType(0), sqlTimestampOf(0, 2001, 1, 22, 3, 4, 0, 0, session));
+        assertFunction("TIMESTAMP '2001-01-22'", createTimestampType(0), sqlTimestampOf(0, 2001, 1, 22, 0, 0, 0, 0, session));
 
-        assertFunction("TIMESTAMP '2001-1-2 3:4:5.321'", TIMESTAMP, sqlTimestampOf(2001, 1, 2, 3, 4, 5, 321, session));
-        assertFunction("TIMESTAMP '2001-1-2 3:4:5'", TIMESTAMP, sqlTimestampOf(2001, 1, 2, 3, 4, 5, 0, session));
-        assertFunction("TIMESTAMP '2001-1-2 3:4'", TIMESTAMP, sqlTimestampOf(2001, 1, 2, 3, 4, 0, 0, session));
-        assertFunction("TIMESTAMP '2001-1-2'", TIMESTAMP, sqlTimestampOf(2001, 1, 2, 0, 0, 0, 0, session));
+        assertFunction("TIMESTAMP '2001-1-2 3:4:5.321'", createTimestampType(3), sqlTimestampOf(3, 2001, 1, 2, 3, 4, 5, 321, session));
+        assertFunction("TIMESTAMP '2001-1-2 3:4:5'", createTimestampType(0), sqlTimestampOf(0, 2001, 1, 2, 3, 4, 5, 0, session));
+        assertFunction("TIMESTAMP '2001-1-2 3:4'", createTimestampType(0), sqlTimestampOf(0, 2001, 1, 2, 3, 4, 0, 0, session));
+        assertFunction("TIMESTAMP '2001-1-2'", createTimestampType(0), sqlTimestampOf(0, 2001, 1, 2, 0, 0, 0, 0, session));
 
         assertInvalidFunction("TIMESTAMP 'text'", INVALID_LITERAL, "line 1:1: 'text' is not a valid timestamp literal");
     }
@@ -208,23 +209,23 @@ public void testCastToTimestampWithTimeZone()
     public void testCastToSlice()
     {
         assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05.321' as varchar)", VARCHAR, "2001-01-22 03:04:05.321");
-        assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05' as varchar)", VARCHAR, "2001-01-22 03:04:05.000");
-        assertFunction("cast(TIMESTAMP '2001-1-22 03:04' as varchar)", VARCHAR, "2001-01-22 03:04:00.000");
-        assertFunction("cast(TIMESTAMP '2001-1-22' as varchar)", VARCHAR, "2001-01-22 00:00:00.000");
+        assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05' as varchar)", VARCHAR, "2001-01-22 03:04:05");
+        assertFunction("cast(TIMESTAMP '2001-1-22 03:04' as varchar)", VARCHAR, "2001-01-22 03:04:00");
+        assertFunction("cast(TIMESTAMP '2001-1-22' as varchar)", VARCHAR, "2001-01-22 00:00:00");
     }
 
     @Test
     public void testCastToJson()
     {
         assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05.321' as json)", JSON, "\"2001-01-22 03:04:05.321\"");
-        assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05' as json)", JSON, "\"2001-01-22 03:04:05.000\"");
-        assertFunction("cast(TIMESTAMP '2001-1-22 03:04' as json)", JSON, "\"2001-01-22 03:04:00.000\"");
-        assertFunction("cast(TIMESTAMP '2001-1-22' as json)", JSON, "\"2001-01-22 00:00:00.000\"");
+        assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05' as json)", JSON, "\"2001-01-22 03:04:05\"");
+        assertFunction("cast(TIMESTAMP '2001-1-22 03:04' as json)", JSON, "\"2001-01-22 03:04:00\"");
+        assertFunction("cast(TIMESTAMP '2001-1-22' as json)", JSON, "\"2001-01-22 00:00:00\"");
 
         assertFunction("cast(ARRAY[TIMESTAMP '2001-1-22 03:04:05.321'] as json)", JSON, "[\"2001-01-22 03:04:05.321\"]");
-        assertFunction("cast(ARRAY[TIMESTAMP '2001-1-22 03:04:05'] as json)", JSON, "[\"2001-01-22 03:04:05.000\"]");
-        assertFunction("cast(ARRAY[TIMESTAMP '2001-1-22 03:04'] as json)", JSON, "[\"2001-01-22 03:04:00.000\"]");
-        assertFunction("cast(ARRAY[TIMESTAMP '2001-1-22'] as json)", JSON, "[\"2001-01-22 00:00:00.000\"]");
+        assertFunction("cast(ARRAY[TIMESTAMP '2001-1-22 03:04:05'] as json)", JSON, "[\"2001-01-22 03:04:05\"]");
+        assertFunction("cast(ARRAY[TIMESTAMP '2001-1-22 03:04'] as json)", JSON, "[\"2001-01-22 03:04:00\"]");
+        assertFunction("cast(ARRAY[TIMESTAMP '2001-1-22'] as json)", JSON, "[\"2001-01-22 00:00:00\"]");
     }
 
     @Test
@@ -232,47 +233,47 @@ public void testCastFromSlice()
     {
         assertFunction("cast('2001-1-22 03:04:05.321' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 321, session));
         assertFunction("cast('2001-1-22 03:04:05' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 5, 0, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 0, session));
         assertFunction("cast('2001-1-22 03:04' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 0, 0, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 0, 0, session));
         assertFunction("cast('2001-1-22' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 0, 0, 0, 0, session));
+                sqlTimestampOf(3, 2001, 1, 22, 0, 0, 0, 0, session));
         assertFunction("cast('\n\t 2001-1-22 03:04:05.321' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 321, session));
         assertFunction("cast('2001-1-22 03:04:05.321 \t\n' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 321, session));
         assertFunction("cast('\n\t 2001-1-22 03:04:05.321 \t\n' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 321, session));
     }
 
     @Test
     public void testGreatest()
     {
         assertFunction("greatest(TIMESTAMP '2013-03-30 01:05', TIMESTAMP '2012-03-30 01:05')",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 30, 1, 5, 0, 0, session));
+                createTimestampType(0),
+                sqlTimestampOf(0, 2013, 3, 30, 1, 5, 0, 0, session));
         assertFunction("greatest(TIMESTAMP '2013-03-30 01:05', TIMESTAMP '2012-03-30 01:05', TIMESTAMP '2012-05-01 01:05')",
-                TIMESTAMP,
-                sqlTimestampOf(2013, 3, 30, 1, 5, 0, 0, session));
+                createTimestampType(0),
+                sqlTimestampOf(0, 2013, 3, 30, 1, 5, 0, 0, session));
     }
 
     @Test
     public void testLeast()
     {
         assertFunction("least(TIMESTAMP '2013-03-30 01:05', TIMESTAMP '2012-03-30 01:05')",
-                TIMESTAMP,
-                sqlTimestampOf(2012, 3, 30, 1, 5, 0, 0, session));
+                createTimestampType(0),
+                sqlTimestampOf(0, 2012, 3, 30, 1, 5, 0, 0, session));
         assertFunction("least(TIMESTAMP '2013-03-30 01:05', TIMESTAMP '2012-03-30 01:05', TIMESTAMP '2012-05-01 01:05')",
-                TIMESTAMP,
-                sqlTimestampOf(2012, 3, 30, 1, 5, 0, 0, session));
+                createTimestampType(0),
+                sqlTimestampOf(0, 2012, 3, 30, 1, 5, 0, 0, session));
     }
 
     @Test
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimestampLegacy.java b/presto-main/src/test/java/io/prestosql/type/TestTimestampLegacy.java
index 24e4b77df2e8b..9cbbec815fb5c 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimestampLegacy.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimestampLegacy.java
@@ -19,6 +19,7 @@
 
 import static io.prestosql.spi.StandardErrorCode.INVALID_LITERAL;
 import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
+import static io.prestosql.spi.type.TimestampType.createTimestampType;
 import static io.prestosql.testing.DateTimeTestingUtils.sqlTimestampOf;
 import static io.prestosql.testing.TestingSession.testSessionBuilder;
 import static io.prestosql.testing.assertions.PrestoExceptionAssert.assertPrestoExceptionThrownBy;
@@ -37,37 +38,37 @@ public void testCastFromSlice()
         super.testCastFromSlice();
         assertFunction(
                 "cast('2001-1-22 03:04:05.321 +07:09' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 8, 55, 5, 321, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 8, 55, 5, 321, session));
         assertFunction(
                 "cast('2001-1-22 03:04:05 +07:09' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 8, 55, 5, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 8, 55, 5, 0, session));
         assertFunction(
                 "cast('2001-1-22 03:04 +07:09' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 8, 55, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 8, 55, 0, 0, session));
         assertFunction(
                 "cast('2001-1-22 +07:09' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 5, 51, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 5, 51, 0, 0, session));
 
         assertFunction(
                 "cast('2001-1-22 03:04:05.321 Asia/Oral' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 12, 4, 5, 321, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 12, 4, 5, 321, session));
         assertFunction(
                 "cast('2001-1-22 03:04:05 Asia/Oral' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 12, 4, 5, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 12, 4, 5, 0, session));
         assertFunction(
                 "cast('2001-1-22 03:04 Asia/Oral' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 12, 4, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 12, 4, 0, 0, session));
         assertFunction(
                 "cast('2001-1-22 Asia/Oral' as timestamp)",
-                TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21, 9, 0, 0, 0, session));
+                createTimestampType(3),
+                sqlTimestampOf(3, 2001, 1, 21, 9, 0, 0, 0, session));
     }
 
     @Test
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZone.java b/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZone.java
index 668dcd6ed3778..3da448c463a85 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZone.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZone.java
@@ -62,11 +62,11 @@ public void testCastToTimestamp()
     {
         assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05.321 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 321, session));
 
         // This TZ had switch in 2014, so if we test for 2014 and used unpacked value we would use wrong shift
         assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05.321 Pacific/Bougainville' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 3, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 3, 4, 5, 321, session));
     }
 }
diff --git a/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZoneLegacy.java b/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZoneLegacy.java
index 99e9f8a348467..8beb56a923aeb 100644
--- a/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZoneLegacy.java
+++ b/presto-main/src/test/java/io/prestosql/type/TestTimestampWithTimeZoneLegacy.java
@@ -43,11 +43,11 @@ public void testCastToTimestamp()
     {
         assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05.321 +07:09' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 22, 2 /* not 3 */, 4, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 22, 2 /* not 3 */, 4, 5, 321, session));
 
         // This TZ had switch in 2014
         assertFunction("cast(TIMESTAMP '2001-1-22 03:04:05.321 Pacific/Bougainville' as timestamp)",
                 TIMESTAMP,
-                sqlTimestampOf(2001, 1, 21 /* not 22 */, 23 /* not 3 */, 13, 5, 321, session));
+                sqlTimestampOf(3, 2001, 1, 21 /* not 22 */, 23 /* not 3 */, 13, 5, 321, session));
     }
 }
diff --git a/presto-oracle/src/main/java/io/prestosql/plugin/oracle/OracleClient.java b/presto-oracle/src/main/java/io/prestosql/plugin/oracle/OracleClient.java
index 84f7db65ec03b..14f3be659711f 100644
--- a/presto-oracle/src/main/java/io/prestosql/plugin/oracle/OracleClient.java
+++ b/presto-oracle/src/main/java/io/prestosql/plugin/oracle/OracleClient.java
@@ -31,7 +31,6 @@
 import io.prestosql.spi.type.Chars;
 import io.prestosql.spi.type.DecimalType;
 import io.prestosql.spi.type.Decimals;
-import io.prestosql.spi.type.TimestampType;
 import io.prestosql.spi.type.Type;
 import io.prestosql.spi.type.VarcharType;
 import oracle.jdbc.OraclePreparedStatement;
@@ -429,7 +428,7 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type)
             }
             return WriteMapping.sliceMapping(dataType, longDecimalWriteFunction((DecimalType) type));
         }
-        if (type instanceof TimestampType) {
+        if (type.equals(TIMESTAMP)) {
             return WriteMapping.longMapping("timestamp(3)", oracleTimestampWriteFunction(session));
         }
         WriteMapping writeMapping = WRITE_MAPPINGS.get(type);
diff --git a/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleDistributedQueries.java b/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleDistributedQueries.java
index 6750d0fa12244..e71b7c33ba44f 100644
--- a/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleDistributedQueries.java
+++ b/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleDistributedQueries.java
@@ -340,7 +340,7 @@ public void testShowColumns()
                 .row("custkey", "decimal(19,0)", "", "")
                 .row("orderstatus", "varchar(1)", "", "")
                 .row("totalprice", "double", "", "")
-                .row("orderdate", "timestamp", "", "")
+                .row("orderdate", "timestamp(3)", "", "")
                 .row("orderpriority", "varchar(15)", "", "")
                 .row("clerk", "varchar(15)", "", "")
                 .row("shippriority", "decimal(10,0)", "", "")
diff --git a/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleIntegrationSmokeTest.java b/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleIntegrationSmokeTest.java
index af6c19880b61d..a7f957562da98 100644
--- a/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleIntegrationSmokeTest.java
+++ b/presto-oracle/src/test/java/io/prestosql/plugin/oracle/TestOracleIntegrationSmokeTest.java
@@ -53,7 +53,7 @@ public void testDescribeTable()
                 .row("custkey", "decimal(19,0)", "", "")
                 .row("orderstatus", "varchar(1)", "", "")
                 .row("totalprice", "double", "", "")
-                .row("orderdate", "timestamp", "", "")
+                .row("orderdate", "timestamp(3)", "", "")
                 .row("orderpriority", "varchar(15)", "", "")
                 .row("clerk", "varchar(15)", "", "")
                 .row("shippriority", "decimal(10,0)", "", "")
@@ -74,7 +74,7 @@ public void testShowCreateTable()
                         "   custkey decimal(19, 0),\n" +
                         "   orderstatus varchar(1),\n" +
                         "   totalprice double,\n" +
-                        "   orderdate timestamp,\n" +
+                        "   orderdate timestamp(3),\n" +
                         "   orderpriority varchar(15),\n" +
                         "   clerk varchar(15),\n" +
                         "   shippriority decimal(10, 0),\n" +
diff --git a/presto-orc/src/test/java/io/prestosql/orc/AbstractTestOrcReader.java b/presto-orc/src/test/java/io/prestosql/orc/AbstractTestOrcReader.java
index 476ecf4ad0e29..43ef7f2b509f9 100644
--- a/presto-orc/src/test/java/io/prestosql/orc/AbstractTestOrcReader.java
+++ b/presto-orc/src/test/java/io/prestosql/orc/AbstractTestOrcReader.java
@@ -296,11 +296,11 @@ public void testLegacyTimestamp()
     {
         @SuppressWarnings("deprecation")
         List values = ImmutableList.of(
-                new SqlTimestamp(0, TimeZoneKey.UTC_KEY),
-                new SqlTimestamp(10, TimeZoneKey.UTC_KEY),
-                new SqlTimestamp(1123456789L, TimeZoneKey.UTC_KEY), // 1970-01-14T00:04:16.789Z
-                new SqlTimestamp(1000123456789L, TimeZoneKey.UTC_KEY), // 2001-09-10T12:04:16.789Z
-                new SqlTimestamp(1575553299564L, TimeZoneKey.UTC_KEY)); // 2019-12-05T13:41:39.564Z
+                SqlTimestamp.legacyFromMillis(3, 0, TimeZoneKey.UTC_KEY),
+                SqlTimestamp.legacyFromMillis(3, 10, TimeZoneKey.UTC_KEY),
+                SqlTimestamp.legacyFromMillis(3, 1123456789L, TimeZoneKey.UTC_KEY), // 1970-01-14T00:04:16.789Z
+                SqlTimestamp.legacyFromMillis(3, 1000123456789L, TimeZoneKey.UTC_KEY), // 2001-09-10T12:04:16.789Z
+                SqlTimestamp.legacyFromMillis(3, 1575553299564L, TimeZoneKey.UTC_KEY)); // 2019-12-05T13:41:39.564Z
         tester.testRoundTrip(TIMESTAMP, newArrayList(limit(cycle(values), 30_000)));
     }
 
diff --git a/presto-orc/src/test/java/io/prestosql/orc/BenchmarkColumnReaders.java b/presto-orc/src/test/java/io/prestosql/orc/BenchmarkColumnReaders.java
index 7bf401e64cbc1..764d29c9f57ae 100644
--- a/presto-orc/src/test/java/io/prestosql/orc/BenchmarkColumnReaders.java
+++ b/presto-orc/src/test/java/io/prestosql/orc/BenchmarkColumnReaders.java
@@ -1035,7 +1035,7 @@ protected Iterator createValues()
         {
             List values = new ArrayList<>();
             for (int i = 0; i < ROWS; ++i) {
-                values.add(new SqlTimestamp((random.nextLong()), UTC_KEY));
+                values.add(SqlTimestamp.legacyFromMillis(3, (random.nextLong()), UTC_KEY));
             }
             return values.iterator();
         }
@@ -1058,7 +1058,7 @@ protected Iterator createValues()
             List values = new ArrayList<>();
             for (int i = 0; i < ROWS; ++i) {
                 if (random.nextBoolean()) {
-                    values.add(new SqlTimestamp(random.nextLong(), UTC_KEY));
+                    values.add(SqlTimestamp.legacyFromMillis(3, random.nextLong(), UTC_KEY));
                 }
                 else {
                     values.add(null);
diff --git a/presto-orc/src/test/java/io/prestosql/orc/TestReadBloomFilter.java b/presto-orc/src/test/java/io/prestosql/orc/TestReadBloomFilter.java
index 8bd5c70e1c032..5dc0c7b4973ed 100644
--- a/presto-orc/src/test/java/io/prestosql/orc/TestReadBloomFilter.java
+++ b/presto-orc/src/test/java/io/prestosql/orc/TestReadBloomFilter.java
@@ -68,7 +68,7 @@ public void test()
 
         testType(DATE, ImmutableList.of(new SqlDate(1), new SqlDate(5_000), new SqlDate(10_000)), 5_000L, 7_777L);
         testType(TIMESTAMP,
-                ImmutableList.of(new SqlTimestamp(1, TIME_ZONE), new SqlTimestamp(500_000L, TIME_ZONE), new SqlTimestamp(1_000_000L, TIME_ZONE)),
+                ImmutableList.of(SqlTimestamp.legacyFromMillis(3, 1, TIME_ZONE), SqlTimestamp.legacyFromMillis(3, 500_000L, TIME_ZONE), SqlTimestamp.legacyFromMillis(3, 1_000_000L, TIME_ZONE)),
                 500_000L + HIVE_STORAGE_TIME_ZONE.getOffset(500_000L),
                 777_777L + HIVE_STORAGE_TIME_ZONE.getOffset(777_777L));
 
diff --git a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAllDatatypesFromHiveConnector.java b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAllDatatypesFromHiveConnector.java
index afd557d6c1628..b35fcdee2d4ef 100644
--- a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAllDatatypesFromHiveConnector.java
+++ b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestAllDatatypesFromHiveConnector.java
@@ -244,7 +244,7 @@ public void testSelectAllDatatypesAvro()
                 row("c_double", "double"),
                 row("c_decimal", "decimal(10,0)"),
                 row("c_decimal_w_params", "decimal(10,5)"),
-                row("c_timestamp", "timestamp"),
+                row("c_timestamp", "timestamp(3)"),
                 row("c_date", "date"),
                 row("c_string", "varchar"),
                 row("c_varchar", "varchar(10)"),
@@ -299,7 +299,7 @@ private void assertProperAllDatatypesSchema(String tableName)
                 row("c_double", "double"),
                 row("c_decimal", "decimal(10,0)"),
                 row("c_decimal_w_params", "decimal(10,5)"),
-                row("c_timestamp", "timestamp"),
+                row("c_timestamp", "timestamp(3)"),
                 row("c_date", "date"),
                 row("c_string", "varchar"),
                 row("c_varchar", "varchar(10)"),
@@ -379,7 +379,7 @@ public void testSelectAllDatatypesParquetFile()
                 row("c_double", "double"),
                 row("c_decimal", "decimal(10,0)"),
                 row("c_decimal_w_params", "decimal(10,5)"),
-                row("c_timestamp", "timestamp"),
+                row("c_timestamp", "timestamp(3)"),
                 row("c_string", "varchar"),
                 row("c_varchar", "varchar(10)"),
                 row("c_char", "char(10)"),
diff --git a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestHiveTableStatistics.java b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestHiveTableStatistics.java
index 58c25db48d9e1..e3a36234d53df 100644
--- a/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestHiveTableStatistics.java
+++ b/presto-product-tests/src/main/java/io/prestosql/tests/hive/TestHiveTableStatistics.java
@@ -1143,7 +1143,7 @@ public void testComputePartitionStatisticsOnCreateTable()
                     "           REAL '123.340', DOUBLE '234.560', " +
                     "           CAST(343.0 AS DECIMAL(10, 0)), " +
                     "           CAST(345.670 AS DECIMAL(10, 5)), " +
-                    "           TIMESTAMP '2015-05-10 12:15:30', " +
+                    "           TIMESTAMP '2015-05-10 12:15:30.000', " +
                     "           DATE '2015-05-08', " +
                     "           CAST('p1 varchar' AS VARCHAR), " +
                     "           CAST('p1 varchar10' AS VARCHAR(10)), " +
@@ -1162,7 +1162,7 @@ public void testComputePartitionStatisticsOnCreateTable()
                     "           DOUBLE '777.560', " +
                     "           CAST(888.0 AS DECIMAL(10, 0)), " +
                     "           CAST(999.670 AS DECIMAL(10, 5)), " +
-                    "           TIMESTAMP '2015-05-10 12:45:30', " +
+                    "           TIMESTAMP '2015-05-10 12:45:30.000', " +
                     "           DATE '2015-05-09', " +
                     "           CAST('p2 varchar' AS VARCHAR), " +
                     "           CAST('p2 varchar10' AS VARCHAR(10)), " +
diff --git a/presto-product-tests/src/main/java/io/prestosql/tests/kafka/TestKafkaSmokeTest.java b/presto-product-tests/src/main/java/io/prestosql/tests/kafka/TestKafkaSmokeTest.java
index aa587e78bb9bd..5b42918a6e29f 100644
--- a/presto-product-tests/src/main/java/io/prestosql/tests/kafka/TestKafkaSmokeTest.java
+++ b/presto-product-tests/src/main/java/io/prestosql/tests/kafka/TestKafkaSmokeTest.java
@@ -291,11 +291,11 @@ public void testSelectAllJsonTable()
                 row("c_tinyint", "tinyint"),
                 row("c_double", "double"),
                 row("c_boolean", "boolean"),
-                row("c_timestamp_milliseconds_since_epoch", "timestamp"),
-                row("c_timestamp_seconds_since_epoch", "timestamp"),
-                row("c_timestamp_iso8601", "timestamp"),
-                row("c_timestamp_rfc2822", "timestamp"),
-                row("c_timestamp_custom", "timestamp"),
+                row("c_timestamp_milliseconds_since_epoch", "timestamp(3)"),
+                row("c_timestamp_seconds_since_epoch", "timestamp(3)"),
+                row("c_timestamp_iso8601", "timestamp(3)"),
+                row("c_timestamp_rfc2822", "timestamp(3)"),
+                row("c_timestamp_custom", "timestamp(3)"),
                 row("c_date_iso8601", "date"),
                 row("c_date_rfc2822", "date"),
                 row("c_date_custom", "date"),
diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result b/presto-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result
index 95f62c53e95cd..932f6ae7e8575 100644
--- a/presto-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result
+++ b/presto-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result
@@ -6,86 +6,85 @@
  date_add | date | varchar(x), bigint, date | scalar | true | Add the specified amount of date to the given date |
  date_add | time | varchar(x), bigint, time | scalar | true | Add the specified amount of time to the given time |
  date_add | time with time zone | varchar(x), bigint, time with time zone | scalar | true | Add the specified amount of time to the given time |
- date_add | timestamp | varchar(x), bigint, timestamp | scalar | true | Add the specified amount of time to the given timestamp |
+ date_add | timestamp(p) | varchar(x), bigint, timestamp(p) | scalar | true | Add the specified amount of time to the given timestamp |
  date_add | timestamp with time zone | varchar(x), bigint, timestamp with time zone | scalar | true | Add the specified amount of time to the given timestamp |
  date_diff | bigint | varchar(x), date, date | scalar | true | Difference of the given dates in the given unit |
  date_diff | bigint | varchar(x), time with time zone, time with time zone | scalar | true | Difference of the given times in the given unit |
  date_diff | bigint | varchar(x), time, time | scalar | true | Difference of the given times in the given unit |
  date_diff | bigint | varchar(x), timestamp with time zone, timestamp with time zone | scalar | true | Difference of the given times in the given unit |
- date_diff | bigint | varchar(x), timestamp, timestamp | scalar | true | Difference of the given times in the given unit |
+ date_diff | bigint | varchar(x), timestamp(p), timestamp(p) | scalar | true | Difference of the given times in the given unit |
  date_format | varchar | timestamp with time zone, varchar(x) | scalar | true | |
- date_format | varchar | timestamp, varchar(x) | scalar | true | |
- date_parse | timestamp | varchar(x), varchar(y) | scalar | true | |
+ date_format | varchar | timestamp(p), varchar(x) | scalar | true | Formats the given timestamp by the given format |
+ date_parse | timestamp(3) | varchar(x), varchar(y) | scalar | true | |
  date_trunc | date | varchar(x), date | scalar | true | Truncate to the specified precision in the session timezone |
  date_trunc | time | varchar(x), time | scalar | true | Truncate to the specified precision in the session timezone |
  date_trunc | time with time zone | varchar(x), time with time zone | scalar | true | Truncate to the specified precision |
- date_trunc | timestamp | varchar(x), timestamp | scalar | true | Truncate to the specified precision in the session timezone |
+ date_trunc | timestamp(p) | varchar(x), timestamp(p) | scalar | true | Truncate to the specified precision in the session timezone |
  date_trunc | timestamp with time zone | varchar(x), timestamp with time zone | scalar | true | Truncate to the specified precision |
  day | bigint | date | scalar | true | Day of the month of the given date |
  day | bigint | interval day to second | scalar | true | Day of the month of the given interval |
- day | bigint | timestamp | scalar | true | Day of the month of the given timestamp |
+ day | bigint | timestamp(p) | scalar | true | Day of the month of the given timestamp |
  day | bigint | timestamp with time zone | scalar | true | Day of the month of the given timestamp |
  day_of_month | bigint | date | scalar | true | Day of the month of the given date |
  day_of_month | bigint | interval day to second | scalar | true | Day of the month of the given interval |
- day_of_month | bigint | timestamp | scalar | true | Day of the month of the given timestamp |
+ day_of_month | bigint | timestamp(p) | scalar | true | Day of the month of the given timestamp |
  day_of_month | bigint | timestamp with time zone | scalar | true | Day of the month of the given timestamp |
  day_of_week | bigint | date | scalar | true | Day of the week of the given date |
- day_of_week | bigint | timestamp | scalar | true | Day of the week of the given timestamp |
+ day_of_week | bigint | timestamp(p) | scalar | true | Day of the week of the given timestamp |
  day_of_week | bigint | timestamp with time zone | scalar | true | Day of the week of the given timestamp |
  day_of_year | bigint | date | scalar | true | Day of the year of the given date |
- day_of_year | bigint | timestamp | scalar | true | Day of the year of the given timestamp |
+ day_of_year | bigint | timestamp(p) | scalar | true | Day of the year of the given timestamp |
  day_of_year | bigint | timestamp with time zone | scalar | true | Day of the year of the given timestamp |
  dow | bigint | date | scalar | true | Day of the week of the given date |
- dow | bigint | timestamp | scalar | true | Day of the week of the given timestamp |
+ dow | bigint | timestamp(p) | scalar | true | Day of the week of the given timestamp |
  dow | bigint | timestamp with time zone | scalar | true | Day of the week of the given timestamp |
  doy | bigint | date | scalar | true | Day of the year of the given date |
- doy | bigint | timestamp | scalar | true | Day of the year of the given timestamp |
+ doy | bigint | timestamp(p) | scalar | true | Day of the year of the given timestamp |
  doy | bigint | timestamp with time zone | scalar | true | Day of the year of the given timestamp |
  e | double | | scalar | true | Euler's number |
  format_datetime | varchar | timestamp with time zone, varchar(x) | scalar | true | Formats the given time by the given format |
- format_datetime | varchar | timestamp, varchar(x) | scalar | true | Formats the given time by the given format |
+ format_datetime | varchar | timestamp(p), varchar(x) | scalar | true | Formats the given time by the given format |
  from_iso8601_date | date | varchar(x) | scalar | true | |
  from_iso8601_timestamp | timestamp with time zone | varchar(x) | scalar | true | |
- from_unixtime | timestamp | double | scalar | true | |
+ from_unixtime | timestamp(3) | double | scalar | true | |
  from_unixtime | timestamp with time zone | double, bigint, bigint | scalar | true | |
  hour | bigint | interval day to second | scalar | true | Hour of the day of the given interval |
  hour | bigint | time | scalar | true | Hour of the day of the given time |
  hour | bigint | time with time zone | scalar | true | Hour of the day of the given time |
- hour | bigint | timestamp | scalar | true | Hour of the day of the given timestamp |
+ hour | bigint | timestamp(p) | scalar | true | Hour of the day of the given timestamp |
  hour | bigint | timestamp with time zone | scalar | true | Hour of the day of the given timestamp |
  localtime | time | | scalar | true | Current time without time zone |
- localtimestamp | timestamp | | scalar | true | Current timestamp without time zone |
  minute | bigint | interval day to second | scalar | true | Minute of the hour of the given interval |
  minute | bigint | time | scalar | true | Minute of the hour of the given time |
  minute | bigint | time with time zone | scalar | true | Minute of the hour of the given time |
- minute | bigint | timestamp | scalar | true | Minute of the hour of the given timestamp |
+ minute | bigint | timestamp(p) | scalar | true | Minute of the hour of the given timestamp |
  minute | bigint | timestamp with time zone | scalar | true | Minute of the hour of the given timestamp |
  month | bigint | date | scalar | true | Month of the year of the given date |
  month | bigint | interval year to month | scalar | true | Month of the year of the given interval |
- month | bigint | timestamp | scalar | true | Month of the year of the given timestamp |
+ month | bigint | timestamp(p) | scalar | true | Month of the year of the given timestamp |
  month | bigint | timestamp with time zone | scalar | true | Month of the year of the given timestamp |
  now | timestamp with time zone | | scalar | true | Current timestamp with time zone |
  parse_datetime | timestamp with time zone | varchar(x), varchar(y) | scalar | true | Parses the specified date/time by the given format |
  quarter | bigint | date | scalar | true | Quarter of the year of the given date |
- quarter | bigint | timestamp | scalar | true | Quarter of the year of the given timestamp |
+ quarter | bigint | timestamp(p) | scalar | true | Quarter of the year of the given timestamp |
  quarter | bigint | timestamp with time zone | scalar | true | Quarter of the year of the given timestamp |
  timezone_hour | bigint | timestamp with time zone | scalar | true | Time zone hour of the given timestamp |
  timezone_minute | bigint | timestamp with time zone | scalar | true | Time zone minute of the given timestamp |
- to_unixtime | double | timestamp | scalar | true | |
+ to_unixtime | double | timestamp(p) | scalar | true | |
  to_unixtime | double | timestamp with time zone | scalar | true | |
  week | bigint | date | scalar | true | Week of the year of the given date |
- week | bigint | timestamp | scalar | true | Week of the year of the given timestamp |
+ week | bigint | timestamp(p) | scalar | true | Week of the year of the given timestamp |
  week | bigint | timestamp with time zone | scalar | true | Week of the year of the given timestamp |
  week_of_year | bigint | date | scalar | true | Week of the year of the given date |
- week_of_year | bigint | timestamp | scalar | true | Week of the year of the given timestamp |
+ week_of_year | bigint | timestamp(p) | scalar | true | Week of the year of the given timestamp |
  week_of_year | bigint | timestamp with time zone | scalar | true | Week of the year of the given timestamp |
  year | bigint | date | scalar | true | Year of the given date |
  year | bigint | interval year to month | scalar | true | Year of the given interval |
- year | bigint | timestamp | scalar | true | Year of the given timestamp |
+ year | bigint | timestamp(p) | scalar | true | Year of the given timestamp |
  year | bigint | timestamp with time zone | scalar | true | Year of the given timestamp |
  year_of_week | bigint | date | scalar | true | Year of the ISO week of the given date |
- year_of_week | bigint | timestamp | scalar | true | Year of the ISO week of the given timestamp |
+ year_of_week | bigint | timestamp(p) | scalar | true | Year of the ISO week of the given timestamp |
  year_of_week | bigint | timestamp with time zone | scalar | true | Year of the ISO week of the given timestamp |
  yow | bigint | date | scalar | true | Year of the ISO week of the given date |
- yow | bigint | timestamp | scalar | true | Year of the ISO week of the given timestamp |
+ yow | bigint | timestamp(p) | scalar | true | Year of the ISO week of the given timestamp |
  yow | bigint | timestamp with time zone | scalar | true | Year of the ISO week of the given timestamp |
diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/system/selectInformationSchemaColumns.result b/presto-product-tests/src/main/resources/sql-tests/testcases/system/selectInformationSchemaColumns.result
index c76581b13d7c0..8b5d9b06abe6b 100644
--- a/presto-product-tests/src/main/resources/sql-tests/testcases/system/selectInformationSchemaColumns.result
+++ b/presto-product-tests/src/main/resources/sql-tests/testcases/system/selectInformationSchemaColumns.result
@@ -79,10 +79,10 @@ system| runtime| queries| resource_group_id| array(varchar)| YES| null| null|
 system| runtime| queries| queued_time_ms| bigint| YES| null| null|
 system| runtime| queries| analysis_time_ms| bigint| YES| null| null|
 system| runtime| queries| planning_time_ms| bigint| YES| null| null|
-system| runtime| queries| created| timestamp| YES| null| null|
-system| runtime| queries| started| timestamp| YES| null| null|
-system| runtime| queries| last_heartbeat| timestamp| YES| null| null|
-system| runtime| queries| end| timestamp| YES| null| null|
+system| runtime| queries| created| timestamp(3)| YES| null| null|
+system| runtime| queries| started| timestamp(3)| YES| null| null|
+system| runtime| queries| last_heartbeat| timestamp(3)| YES| null| null|
+system| runtime| queries| end| timestamp(3)| YES| null| null|
 system| runtime| queries| error_type| varchar| YES| null| null|
 system| runtime| queries| error_code| varchar| YES| null| null|
 system| runtime| tasks| node_id| varchar| YES| null| null|
@@ -105,15 +105,15 @@ system| runtime| tasks| output_bytes| bigint| YES| null| null|
 system| runtime| tasks| output_rows| bigint| YES| null| null|
 system| runtime| tasks| physical_input_bytes| bigint| YES| null| null|
 system| runtime| tasks| physical_written_bytes| bigint| YES| null| null|
-system| runtime| tasks| created| timestamp| YES| null| null|
-system| runtime| tasks| start| timestamp| YES| null| null|
-system| runtime| tasks| last_heartbeat| timestamp| YES| null| null|
-system| runtime| tasks| end| timestamp| YES| null| null|
+system| runtime| tasks| created| timestamp(3)| YES| null| null|
+system| runtime| tasks| start| timestamp(3)| YES| null| null|
+system| runtime| tasks| last_heartbeat| timestamp(3)| YES| null| null|
+system| runtime| tasks| end| timestamp(3)| YES| null| null|
 system| runtime| transactions| transaction_id| varchar| YES| null| null|
 system| runtime| transactions| isolation_level| varchar| YES| null| null|
 system| runtime| transactions| read_only| boolean| YES| null| null|
 system| runtime| transactions| auto_commit_context| boolean| YES| null| null|
-system| runtime| transactions| create_time| timestamp| YES| null| null|
+system| runtime| transactions| create_time| timestamp(3)| YES| null| null|
 system| runtime| transactions| idle_time_secs| bigint| YES| null| null|
 system| runtime| transactions| written_catalog| varchar| YES| null| null|
 system| runtime| transactions| catalogs| array(varchar)| YES| null| null|
diff --git a/presto-prometheus/src/test/java/io/prestosql/plugin/prometheus/MetadataUtil.java b/presto-prometheus/src/test/java/io/prestosql/plugin/prometheus/MetadataUtil.java
index fb2a66f32b899..e931d702da8b8 100644
--- a/presto-prometheus/src/test/java/io/prestosql/plugin/prometheus/MetadataUtil.java
+++ b/presto-prometheus/src/test/java/io/prestosql/plugin/prometheus/MetadataUtil.java
@@ -62,12 +62,14 @@ private MetadataUtil() {}
     public static final class TestingTypeDeserializer
             extends FromStringDeserializer
     {
-        private final Map types = ImmutableMap.of(
-                varcharMapType.getTypeSignature().toString(), varcharMapType,
-                StandardTypes.BIGINT, BIGINT,
-                StandardTypes.TIMESTAMP, TIMESTAMP,
-                StandardTypes.DOUBLE, DOUBLE,
-                StandardTypes.VARCHAR, createUnboundedVarcharType());
+        private final Map types = ImmutableMap.builder()
+                .put(varcharMapType.getTypeSignature().toString(), varcharMapType)
+                .put(StandardTypes.BIGINT, BIGINT)
+                .put(StandardTypes.TIMESTAMP, TIMESTAMP)
+                .put("timestamp(3)", TIMESTAMP)
+                .put(StandardTypes.DOUBLE, DOUBLE)
+                .put(StandardTypes.VARCHAR, createUnboundedVarcharType())
+                .build();
 
         public TestingTypeDeserializer()
         {
diff --git a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorConnector.java b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorConnector.java
index 48642b82114d0..2c37f0622a4e4 100644
--- a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorConnector.java
+++ b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorConnector.java
@@ -61,6 +61,7 @@
 import static com.google.common.io.MoreFiles.deleteRecursively;
 import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE;
 import static io.prestosql.metadata.MetadataManager.createTestMetadataManager;
+import static io.prestosql.operator.scalar.timestamp.VarcharToTimestampCast.castToLegacyShortTimestamp;
 import static io.prestosql.plugin.raptor.legacy.RaptorTableProperties.TEMPORAL_COLUMN_PROPERTY;
 import static io.prestosql.plugin.raptor.legacy.metadata.SchemaDaoUtil.createTablesWithRetry;
 import static io.prestosql.plugin.raptor.legacy.metadata.TestDatabaseShardManager.createShardManager;
@@ -71,7 +72,6 @@
 import static io.prestosql.spi.type.TimeZoneKey.getTimeZoneKey;
 import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
 import static io.prestosql.testing.TestingConnectorSession.SESSION;
-import static io.prestosql.util.DateTimeUtils.convertToLegacyTimestamp;
 import static io.prestosql.util.DateTimeUtils.parseDate;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertFalse;
@@ -237,8 +237,8 @@ private void assertSplitShard(Type temporalType, String min, String max, String
         Object timestamp1 = null;
         Object timestamp2 = null;
         if (temporalType.equals(TIMESTAMP)) {
-            timestamp1 = new SqlTimestamp(convertToLegacyTimestamp(getTimeZoneKey(userTimeZone), min), getTimeZoneKey(userTimeZone));
-            timestamp2 = new SqlTimestamp(convertToLegacyTimestamp(getTimeZoneKey(userTimeZone), max), getTimeZoneKey(userTimeZone));
+            timestamp1 = SqlTimestamp.legacyFromMillis(3, castToLegacyShortTimestamp(TIMESTAMP.getPrecision(), getTimeZoneKey(userTimeZone), min), getTimeZoneKey(userTimeZone));
+            timestamp2 = SqlTimestamp.legacyFromMillis(3, castToLegacyShortTimestamp(TIMESTAMP.getPrecision(), getTimeZoneKey(userTimeZone), max), getTimeZoneKey(userTimeZone));
         }
         else if (temporalType.equals(DATE)) {
             timestamp1 = new SqlDate(parseDate(min));
diff --git a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java
index 661ccac99a6f4..cc7ded9cd58fd 100644
--- a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java
+++ b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/TestRaptorIntegrationSmokeTest.java
@@ -505,7 +505,7 @@ public void testShowCreateTable()
                         "   \"c'4\" array(bigint),\n" +
                         "   c5 map(bigint, varchar),\n" +
                         "   c6 bigint,\n" +
-                        "   c7 timestamp\n" +
+                        "   c7 timestamp(3)\n" +
                         ")\n" +
                         "WITH (\n" +
                         "   bucket_count = 32,\n" +
@@ -534,7 +534,7 @@ public void testShowCreateTable()
                         "   \"c'4\" array(bigint),\n" +
                         "   c5 map(bigint, varchar),\n" +
                         "   c6 bigint,\n" +
-                        "   c7 timestamp\n" +
+                        "   c7 timestamp(3)\n" +
                         ")\n" +
                         "WITH (\n" +
                         "   bucket_count = 32,\n" +
diff --git a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/storage/TestOrcStorageManager.java b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/storage/TestOrcStorageManager.java
index ba0775a2df980..3a488924d83f0 100644
--- a/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/storage/TestOrcStorageManager.java
+++ b/presto-raptor-legacy/src/test/java/io/prestosql/plugin/raptor/legacy/storage/TestOrcStorageManager.java
@@ -691,6 +691,6 @@ private static SqlDate sqlDate(int year, int month, int day)
 
     private static SqlTimestamp sqlTimestamp(int year, int month, int day, int hour, int minute, int second)
     {
-        return sqlTimestampOf(year, month, day, hour, minute, second, 0, UTC, UTC_KEY, SESSION);
+        return sqlTimestampOf(3, year, month, day, hour, minute, second, 0, UTC, UTC_KEY, SESSION);
     }
 }
diff --git a/presto-rcfile/src/test/java/io/prestosql/rcfile/RcFileTester.java b/presto-rcfile/src/test/java/io/prestosql/rcfile/RcFileTester.java
index ca4493509ef78..19a0d4b6811ed 100644
--- a/presto-rcfile/src/test/java/io/prestosql/rcfile/RcFileTester.java
+++ b/presto-rcfile/src/test/java/io/prestosql/rcfile/RcFileTester.java
@@ -847,10 +847,10 @@ else if (actualValue instanceof Text) {
         else if (actualValue instanceof TimestampWritable) {
             TimestampWritable timestamp = (TimestampWritable) actualValue;
             if (SESSION.isLegacyTimestamp()) {
-                actualValue = new SqlTimestamp((timestamp.getSeconds() * 1000) + (timestamp.getNanos() / 1000000L), UTC_KEY);
+                actualValue = SqlTimestamp.legacyFromMillis(3, (timestamp.getSeconds() * 1000) + (timestamp.getNanos() / 1000000L), UTC_KEY);
             }
             else {
-                actualValue = new SqlTimestamp((timestamp.getSeconds() * 1000) + (timestamp.getNanos() / 1000000L));
+                actualValue = SqlTimestamp.fromMillis(3, (timestamp.getSeconds() * 1000) + (timestamp.getNanos() / 1000000L));
             }
         }
         else if (actualValue instanceof StructObject) {
diff --git a/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestCustomDateTimeJsonFieldDecoder.java b/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestCustomDateTimeJsonFieldDecoder.java
index 4655bf8b0a28e..a045d92dc16d3 100644
--- a/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestCustomDateTimeJsonFieldDecoder.java
+++ b/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestCustomDateTimeJsonFieldDecoder.java
@@ -73,11 +73,11 @@ public void testDecodeNulls()
     @Test
     public void testDecodeInvalid()
     {
-        timestampTester.assertInvalidInput("1", TIMESTAMP, "could not parse value '1' as 'timestamp' for column 'some_column'");
-        timestampTester.assertInvalidInput("{}", TIMESTAMP, "could not parse non-value node as 'timestamp' for column 'some_column'");
-        timestampTester.assertInvalidInput("\"a\"", TIMESTAMP, "could not parse value 'a' as 'timestamp' for column 'some_column'");
-        timestampTester.assertInvalidInput("\"15:13:18\"", TIMESTAMP, "could not parse value '15:13:18' as 'timestamp' for column 'some_column'");
-        timestampTester.assertInvalidInput("\"02/2018/11\"", TIMESTAMP, "could not parse value '02/2018/11' as 'timestamp' for column 'some_column'");
+        timestampTester.assertInvalidInput("1", TIMESTAMP, "\\Qcould not parse value '1' as 'timestamp(3)' for column 'some_column'\\E");
+        timestampTester.assertInvalidInput("{}", TIMESTAMP, "\\Qcould not parse non-value node as 'timestamp(3)' for column 'some_column'\\E");
+        timestampTester.assertInvalidInput("\"a\"", TIMESTAMP, "\\Qcould not parse value 'a' as 'timestamp(3)' for column 'some_column'\\E");
+        timestampTester.assertInvalidInput("\"15:13:18\"", TIMESTAMP, "\\Qcould not parse value '15:13:18' as 'timestamp(3)' for column 'some_column'\\E");
+        timestampTester.assertInvalidInput("\"02/2018/11\"", TIMESTAMP, "\\Qcould not parse value '02/2018/11' as 'timestamp(3)' for column 'some_column'\\E");
     }
 
     @Test
diff --git a/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestISO8601JsonFieldDecoder.java b/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestISO8601JsonFieldDecoder.java
index d64734124ae94..9d0927b3aec28 100644
--- a/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestISO8601JsonFieldDecoder.java
+++ b/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestISO8601JsonFieldDecoder.java
@@ -60,10 +60,10 @@ public void testDecodeNulls()
     @Test
     public void testDecodeInvalid()
     {
-        tester.assertInvalidInput("1", TIMESTAMP, "could not parse value '1' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("{}", TIMESTAMP, "could not parse non-value node as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"a\"", TIMESTAMP, "could not parse value 'a' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("1", TIMESTAMP, "could not parse value '1' as 'timestamp' for column 'some_column'");
+        tester.assertInvalidInput("1", TIMESTAMP, "\\Qcould not parse value '1' as 'timestamp(3)' for column 'some_column'\\E");
+        tester.assertInvalidInput("{}", TIMESTAMP, "\\Qcould not parse non-value node as 'timestamp(3)' for column 'some_column'\\E");
+        tester.assertInvalidInput("\"a\"", TIMESTAMP, "\\Qcould not parse value 'a' as 'timestamp(3)' for column 'some_column'\\E");
+        tester.assertInvalidInput("1", TIMESTAMP, "\\Qcould not parse value '1' as 'timestamp(3)' for column 'some_column'\\E");
 
         tester.assertInvalidInput("\"2018-02-19T09:20:11\"", DATE, "could not parse value '2018-02-19T09:20:11' as 'date' for column 'some_column'");
         tester.assertInvalidInput("\"2018-02-19T09:20:11Z\"", DATE, "could not parse value '2018-02-19T09:20:11Z' as 'date' for column 'some_column'");
diff --git a/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestRFC2822JsonFieldDecoder.java b/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestRFC2822JsonFieldDecoder.java
index cf2c690dcbf29..7ca271de9a153 100644
--- a/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestRFC2822JsonFieldDecoder.java
+++ b/presto-record-decoder/src/test/java/io/prestosql/decoder/json/TestRFC2822JsonFieldDecoder.java
@@ -53,16 +53,16 @@ public void testDecodeNulls()
     @Test
     public void testDecodeInvalid()
     {
-        tester.assertInvalidInput("{}", TIMESTAMP, "could not parse non-value node as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"a\"", TIMESTAMP, "could not parse value 'a' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("2018", TIMESTAMP, "could not parse value '2018' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Mon Feb 12 13:15:16 Z\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Mon Feb 12 13:15:16 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Mon Feb 12 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Mon Feb 13:15:16 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Mon 12 13:15:16 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Feb 12 13:15:16 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Fri Feb 09 13:15:19 Europe/Warsaw 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
-        tester.assertInvalidInput("\"Fri Feb 09 13:15:19 EST 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp' for column 'some_column'");
+        tester.assertInvalidInput("{}", TIMESTAMP, "\\Qcould not parse non-value node as 'timestamp(3)' for column 'some_column'\\E");
+        tester.assertInvalidInput("\"a\"", TIMESTAMP, "\\Qcould not parse value 'a' as 'timestamp(3)' for column 'some_column'\\E");
+        tester.assertInvalidInput("2018", TIMESTAMP, "\\Qcould not parse value '2018' as 'timestamp(3)' for column 'some_column'\\E");
+        tester.assertInvalidInput("\"Mon Feb 12 13:15:16 Z\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
+        tester.assertInvalidInput("\"Mon Feb 12 13:15:16 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
+        tester.assertInvalidInput("\"Mon Feb 12 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
+        tester.assertInvalidInput("\"Mon Feb 13:15:16 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
+        tester.assertInvalidInput("\"Mon 12 13:15:16 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
+        tester.assertInvalidInput("\"Feb 12 13:15:16 Z 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
+        tester.assertInvalidInput("\"Fri Feb 09 13:15:19 Europe/Warsaw 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
+        tester.assertInvalidInput("\"Fri Feb 09 13:15:19 EST 2018\"", TIMESTAMP, "could not parse value '.*' as 'timestamp\\(3\\)' for column 'some_column'");
     }
 }
diff --git a/presto-redis/src/test/java/io/prestosql/plugin/redis/util/RedisLoader.java b/presto-redis/src/test/java/io/prestosql/plugin/redis/util/RedisLoader.java
index c54580c49ea1b..0f6cd8dc1b0a0 100644
--- a/presto-redis/src/test/java/io/prestosql/plugin/redis/util/RedisLoader.java
+++ b/presto-redis/src/test/java/io/prestosql/plugin/redis/util/RedisLoader.java
@@ -37,6 +37,7 @@
 import java.util.concurrent.atomic.AtomicReference;
 
 import static com.google.common.base.Preconditions.checkState;
+import static io.prestosql.operator.scalar.timestamp.VarcharToTimestampCast.castToLegacyShortTimestamp;
 import static io.prestosql.spi.type.BigintType.BIGINT;
 import static io.prestosql.spi.type.BooleanType.BOOLEAN;
 import static io.prestosql.spi.type.DateTimeEncoding.unpackMillisUtc;
@@ -47,7 +48,6 @@
 import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
 import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
 import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
-import static io.prestosql.util.DateTimeUtils.convertToLegacyTimestamp;
 import static io.prestosql.util.DateTimeUtils.parseLegacyTime;
 import static java.util.Objects.requireNonNull;
 
@@ -169,7 +169,7 @@ private Object convertValue(Object value, Type type)
                 return ISO8601_FORMATTER.print(parseLegacyTime(timeZoneKey, (String) value));
             }
             if (TIMESTAMP.equals(type)) {
-                return ISO8601_FORMATTER.print(convertToLegacyTimestamp(timeZoneKey, (String) value));
+                return ISO8601_FORMATTER.print(castToLegacyShortTimestamp(TIMESTAMP.getPrecision(), timeZoneKey, (String) value));
             }
             if (TIME_WITH_TIME_ZONE.equals(type) || TIMESTAMP_WITH_TIME_ZONE.equals(type)) {
                 return ISO8601_FORMATTER.print(unpackMillisUtc(DateTimeUtils.convertToTimestampWithTimeZone(timeZoneKey, (String) value)));
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/LongTimestamp.java b/presto-spi/src/main/java/io/prestosql/spi/type/LongTimestamp.java
new file mode 100644
index 0000000000000..7a20f7ff027a5
--- /dev/null
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/LongTimestamp.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.prestosql.spi.type;
+
+import java.util.Objects;
+
+public final class LongTimestamp
+{
+    private static final int PICOSECONDS_PER_MICROSECOND = 1_000_000;
+
+    private final long epochMicros;
+    private final int picosOfMicro; // number of picoSeconds of the microsecond corresponding to epochMicros. It represents an increment towards the positive side.
+
+    public LongTimestamp(long epochMicros, int picosOfMicro)
+    {
+        if (picosOfMicro < 0) {
+            throw new IllegalArgumentException("picosOfMicro must be >= 0");
+        }
+        if (picosOfMicro >= PICOSECONDS_PER_MICROSECOND) {
+            throw new IllegalArgumentException("picosOfMicro must be < 1_000_000");
+        }
+        this.epochMicros = epochMicros;
+        this.picosOfMicro = picosOfMicro;
+    }
+
+    public long getEpochMicros()
+    {
+        return epochMicros;
+    }
+
+    public int getPicosOfMicro()
+    {
+        return picosOfMicro;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        LongTimestamp that = (LongTimestamp) o;
+        return epochMicros == that.epochMicros &&
+                picosOfMicro == that.picosOfMicro;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(epochMicros, picosOfMicro);
+    }
+}
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/LongTimestampType.java b/presto-spi/src/main/java/io/prestosql/spi/type/LongTimestampType.java
new file mode 100644
index 0000000000000..8257e1879aa75
--- /dev/null
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/LongTimestampType.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.prestosql.spi.type;
+
+import io.airlift.slice.XxHash64;
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+import io.prestosql.spi.block.BlockBuilderStatus;
+import io.prestosql.spi.block.Int96ArrayBlockBuilder;
+import io.prestosql.spi.block.PageBuilderStatus;
+import io.prestosql.spi.connector.ConnectorSession;
+
+import static io.airlift.slice.SizeOf.SIZE_OF_LONG;
+
+/**
+ * The representation is a 96-bit value that contains the microseconds from the epoch
+ * in the first long and the fractional increment in the remaining integer, as
+ * a number of picoseconds additional to the epoch microsecond.
+ */
+public class LongTimestampType
+        extends TimestampType
+{
+    public LongTimestampType(int precision)
+    {
+        super(precision, LongTimestamp.class);
+    }
+
+    @Override
+    public int getFixedSize()
+    {
+        return Long.BYTES + Integer.BYTES;
+    }
+
+    @Override
+    public BlockBuilder createBlockBuilder(BlockBuilderStatus blockBuilderStatus, int expectedEntries, int expectedBytesPerEntry)
+    {
+        int maxBlockSizeInBytes;
+        if (blockBuilderStatus == null) {
+            maxBlockSizeInBytes = PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES;
+        }
+        else {
+            maxBlockSizeInBytes = blockBuilderStatus.getMaxPageSizeInBytes();
+        }
+        return new Int96ArrayBlockBuilder(
+                blockBuilderStatus,
+                Math.min(expectedEntries, maxBlockSizeInBytes / getFixedSize()));
+    }
+
+    @Override
+    public BlockBuilder createBlockBuilder(BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+    {
+        return createBlockBuilder(blockBuilderStatus, expectedEntries, getFixedSize());
+    }
+
+    @Override
+    public BlockBuilder createFixedSizeBlockBuilder(int positionCount)
+    {
+        return new Int96ArrayBlockBuilder(null, positionCount);
+    }
+
+    @Override
+    public boolean equalTo(Block leftBlock, int leftPosition, Block rightBlock, int rightPosition)
+    {
+        return compareTo(leftBlock, leftPosition, rightBlock, rightPosition) == 0;
+    }
+
+    @Override
+    public int compareTo(Block leftBlock, int leftPosition, Block rightBlock, int rightPosition)
+    {
+        long leftEpochMicros = getEpochMicros(leftBlock, leftPosition);
+        int leftFraction = getFraction(leftBlock, leftPosition);
+        long rightEpochMicros = getEpochMicros(rightBlock, rightPosition);
+        int rightFraction = getFraction(rightBlock, rightPosition);
+
+        int value = Long.compare(leftEpochMicros, rightEpochMicros);
+        if (value != 0) {
+            return value;
+        }
+        return Integer.compareUnsigned(leftFraction, rightFraction);
+    }
+
+    @Override
+    public long hash(Block block, int position)
+    {
+        return hash(getEpochMicros(block, position), getFraction(block, position));
+    }
+
+    @Override
+    public void appendTo(Block block, int position, BlockBuilder blockBuilder)
+    {
+        if (block.isNull(position)) {
+            blockBuilder.appendNull();
+        }
+        else {
+            blockBuilder.writeLong(getEpochMicros(block, position));
+            blockBuilder.writeInt(getFraction(block, position));
+            blockBuilder.closeEntry();
+        }
+    }
+
+    @Override
+    public Object getObject(Block block, int position)
+    {
+        return new LongTimestamp(getEpochMicros(block, position), getFraction(block, position));
+    }
+
+    @Override
+    public void writeObject(BlockBuilder blockBuilder, Object value)
+    {
+        LongTimestamp timestamp = (LongTimestamp) value;
+        write(blockBuilder, timestamp.getEpochMicros(), timestamp.getPicosOfMicro());
+    }
+
+    public void write(BlockBuilder blockBuilder, long epochMicros, int fraction)
+    {
+        blockBuilder.writeLong(epochMicros);
+        blockBuilder.writeInt(fraction);
+        blockBuilder.closeEntry();
+    }
+
+    @Override
+    public Object getObjectValue(ConnectorSession session, Block block, int position)
+    {
+        if (block.isNull(position)) {
+            return null;
+        }
+
+        long epochMicros = getEpochMicros(block, position);
+        int fraction = getFraction(block, position);
+
+        if (session.isLegacyTimestamp()) {
+            return SqlTimestamp.newLegacyInstance(getPrecision(), epochMicros, fraction, session.getTimeZoneKey());
+        }
+        else {
+            return SqlTimestamp.newInstance(getPrecision(), epochMicros, fraction);
+        }
+    }
+
+    public static long hash(LongTimestamp value)
+    {
+        return hash(value.getEpochMicros(), value.getPicosOfMicro());
+    }
+
+    public static long hash(long epochMicros, long fraction)
+    {
+        return XxHash64.hash(epochMicros) ^ XxHash64.hash(fraction);
+    }
+
+    private static long getEpochMicros(Block block, int position)
+    {
+        return block.getLong(position, 0);
+    }
+
+    private static int getFraction(Block block, int position)
+    {
+        return block.getInt(position, SIZE_OF_LONG);
+    }
+}
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/ShortTimestampType.java b/presto-spi/src/main/java/io/prestosql/spi/type/ShortTimestampType.java
new file mode 100644
index 0000000000000..e4c509327856c
--- /dev/null
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/ShortTimestampType.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.prestosql.spi.type;
+
+import io.prestosql.spi.block.Block;
+import io.prestosql.spi.block.BlockBuilder;
+import io.prestosql.spi.block.BlockBuilderStatus;
+import io.prestosql.spi.block.LongArrayBlockBuilder;
+import io.prestosql.spi.block.PageBuilderStatus;
+import io.prestosql.spi.connector.ConnectorSession;
+
+import static java.lang.Long.rotateLeft;
+import static java.lang.Math.multiplyExact;
+
+/**
+ * Encodes timestamps up to p = 6.
+ *
+ * For 0 <= p <= 3, the value is encoded as milliseconds from the 1970-01-01 00:00:00 epoch.
+ * For 3 < p <= 6, the value is encoded as microseconds from the 1970-01-01 00:00:00 epoch.
+ */
+public class ShortTimestampType
+        extends TimestampType
+{
+    public ShortTimestampType(int precision)
+    {
+        super(precision, long.class);
+    }
+
+    @Override
+    public final int getFixedSize()
+    {
+        return Long.BYTES;
+    }
+
+    @Override
+    public final long getLong(Block block, int position)
+    {
+        return block.getLong(position, 0);
+    }
+
+    @Override
+    public final void writeLong(BlockBuilder blockBuilder, long value)
+    {
+        blockBuilder.writeLong(value).closeEntry();
+    }
+
+    @Override
+    public final void appendTo(Block block, int position, BlockBuilder blockBuilder)
+    {
+        if (block.isNull(position)) {
+            blockBuilder.appendNull();
+        }
+        else {
+            blockBuilder.writeLong(block.getLong(position, 0)).closeEntry();
+        }
+    }
+
+    @Override
+    public boolean equalTo(Block leftBlock, int leftPosition, Block rightBlock, int rightPosition)
+    {
+        long leftValue = leftBlock.getLong(leftPosition, 0);
+        long rightValue = rightBlock.getLong(rightPosition, 0);
+        return leftValue == rightValue;
+    }
+
+    @Override
+    public long hash(Block block, int position)
+    {
+        return hash(block.getLong(position, 0));
+    }
+
+    @Override
+    public int compareTo(Block leftBlock, int leftPosition, Block rightBlock, int rightPosition)
+    {
+        long leftValue = leftBlock.getLong(leftPosition, 0);
+        long rightValue = rightBlock.getLong(rightPosition, 0);
+        return Long.compare(leftValue, rightValue);
+    }
+
+    @Override
+    public final BlockBuilder createBlockBuilder(BlockBuilderStatus blockBuilderStatus, int expectedEntries, int expectedBytesPerEntry)
+    {
+        int maxBlockSizeInBytes;
+        if (blockBuilderStatus == null) {
+            maxBlockSizeInBytes = PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES;
+        }
+        else {
+            maxBlockSizeInBytes = blockBuilderStatus.getMaxPageSizeInBytes();
+        }
+        return new LongArrayBlockBuilder(
+                blockBuilderStatus,
+                Math.min(expectedEntries, maxBlockSizeInBytes / Long.BYTES));
+    }
+
+    @Override
+    public final BlockBuilder createBlockBuilder(BlockBuilderStatus blockBuilderStatus, int expectedEntries)
+    {
+        return createBlockBuilder(blockBuilderStatus, expectedEntries, Long.BYTES);
+    }
+
+    @Override
+    public final BlockBuilder createFixedSizeBlockBuilder(int positionCount)
+    {
+        return new LongArrayBlockBuilder(null, positionCount);
+    }
+
+    @Override
+    public Object getObjectValue(ConnectorSession session, Block block, int position)
+    {
+        if (block.isNull(position)) {
+            return null;
+        }
+
+        long value = block.getLong(position, 0);
+
+        if (getPrecision() <= 3) {
+            value = scaleEpochMillisToMicros(value);
+        }
+
+        if (session.isLegacyTimestamp()) {
+            return SqlTimestamp.newLegacyInstance(getPrecision(), value, 0, session.getTimeZoneKey());
+        }
+        else {
+            return SqlTimestamp.newInstance(getPrecision(), value, 0);
+        }
+    }
+
+    public static long hash(long value)
+    {
+        // xxhash64 mix
+        return rotateLeft(value * 0xC2B2AE3D27D4EB4FL, 31) * 0x9E3779B185EBCA87L;
+    }
+
+    private static long scaleEpochMillisToMicros(long epochMillis)
+    {
+        return multiplyExact(epochMillis, 1000);
+    }
+}
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/SqlTime.java b/presto-spi/src/main/java/io/prestosql/spi/type/SqlTime.java
index 079972a0b572b..d34f0a79d6124 100644
--- a/presto-spi/src/main/java/io/prestosql/spi/type/SqlTime.java
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/SqlTime.java
@@ -44,7 +44,6 @@ public SqlTime(long millisUtc, TimeZoneKey sessionTimeZoneKey)
 
     public long getMillis()
     {
-        checkState(!isLegacyTimestamp(), "getMillis() can be called in new timestamp semantics only");
         return millis;
     }
 
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/SqlTimestamp.java b/presto-spi/src/main/java/io/prestosql/spi/type/SqlTimestamp.java
index 0f30a84d5c871..f4e82bba62ed9 100644
--- a/presto-spi/src/main/java/io/prestosql/spi/type/SqlTimestamp.java
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/SqlTimestamp.java
@@ -16,38 +16,107 @@
 import com.fasterxml.jackson.annotation.JsonValue;
 
 import java.time.Instant;
+import java.time.LocalDateTime;
 import java.time.ZoneId;
 import java.time.ZoneOffset;
 import java.time.format.DateTimeFormatter;
 import java.util.Objects;
 import java.util.Optional;
 
+import static io.prestosql.spi.type.TimestampType.MAX_PRECISION;
+import static java.lang.Math.floorMod;
+import static java.lang.String.format;
+
 public final class SqlTimestamp
 {
     // This needs to be Locale-independent, Java Time's DateTimeFormatter compatible and should never change, as it defines the external API data format.
-    public static final String JSON_FORMAT = "uuuu-MM-dd HH:mm:ss.SSS";
+    public static final String JSON_FORMAT = "uuuu-MM-dd HH:mm:ss[.SSS]";
     public static final DateTimeFormatter JSON_FORMATTER = DateTimeFormatter.ofPattern(JSON_FORMAT);
 
-    private final long millis;
+    private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd HH:mm:ss");
+
+    private static final int MICROSECONDS_PER_SECOND = 1_000_000;
+    private static final int PICOSECONDS_PER_MICROSECOND = 1_000_000;
+
+    private static final long[] POWERS_OF_TEN = {
+            1L,
+            10L,
+            100L,
+            1000L,
+            10_000L,
+            100_000L,
+            1_000_000L,
+            10_000_000L,
+            100_000_000L,
+            1_000_000_000L,
+            10_000_000_000L,
+            100_000_000_000L,
+            1000_000_000_000L
+    };
+
+    private final int precision;
+    private final long epochMicros;
+    private final int picosOfMicros;
     private final Optional sessionTimeZoneKey;
 
-    public SqlTimestamp(long millis)
+    public static SqlTimestamp fromMillis(int precision, long millis)
     {
-        this.millis = millis;
-        sessionTimeZoneKey = Optional.empty();
+        return newInstance(precision, millis * 1000, 0);
     }
 
     @Deprecated
-    public SqlTimestamp(long millisUtc, TimeZoneKey sessionTimeZoneKey)
+    public static SqlTimestamp legacyFromMillis(int precision, long millisUtc, TimeZoneKey sessionTimeZoneKey)
+    {
+        return newLegacyInstance(precision, millisUtc * 1000, 0, sessionTimeZoneKey);
+    }
+
+    public static SqlTimestamp newInstance(int precision, long epochMicros, int picosOfMicro)
+    {
+        return newInstanceWithRounding(precision, epochMicros, picosOfMicro, Optional.empty());
+    }
+
+    @Deprecated
+    public static SqlTimestamp newLegacyInstance(int precision, long epochMicros, int picosOfMicro, TimeZoneKey sessionTimeZoneKey)
+    {
+        return newInstanceWithRounding(precision, epochMicros, picosOfMicro, Optional.of(sessionTimeZoneKey));
+    }
+
+    private static SqlTimestamp newInstanceWithRounding(int precision, long epochMicros, int picosOfMicro, Optional sessionTimeZoneKey)
     {
-        this.millis = millisUtc;
-        this.sessionTimeZoneKey = Optional.of(sessionTimeZoneKey);
+        if (precision < 6) {
+            epochMicros = round(epochMicros, 6 - precision);
+            picosOfMicro = 0;
+        }
+        else if (precision == 6) {
+            if (round(picosOfMicro, 6) == PICOSECONDS_PER_MICROSECOND) {
+                epochMicros++;
+            }
+            picosOfMicro = 0;
+        }
+        else {
+            picosOfMicro = (int) round(picosOfMicro, 12 - precision);
+        }
+
+        return new SqlTimestamp(precision, epochMicros, picosOfMicro, sessionTimeZoneKey);
+    }
+
+    private SqlTimestamp(int precision, long epochMicros, int picosOfMicro, Optional sessionTimeZoneKey)
+    {
+        this.precision = precision;
+        this.epochMicros = epochMicros;
+        this.picosOfMicros = picosOfMicro;
+        this.sessionTimeZoneKey = sessionTimeZoneKey;
+    }
+
+    public int getPrecision()
+    {
+        return precision;
     }
 
     public long getMillis()
     {
         checkState(!isLegacyTimestamp(), "getMillis() can be called in new timestamp semantics only");
-        return millis;
+        return roundDiv(epochMicros, 1000);
     }
 
     /**
@@ -57,7 +126,17 @@ public long getMillis()
     public long getMillisUtc()
     {
         checkState(isLegacyTimestamp(), "getMillisUtc() can be called in legacy timestamp semantics only");
-        return millis;
+        return roundDiv(epochMicros, 1000);
+    }
+
+    public long getEpochMicros()
+    {
+        return epochMicros;
+    }
+
+    public long getPicosOfMicros()
+    {
+        return picosOfMicros;
     }
 
     /**
@@ -74,24 +153,31 @@ public boolean isLegacyTimestamp()
         return sessionTimeZoneKey.isPresent();
     }
 
-    @Override
-    public int hashCode()
+    public SqlTimestamp roundTo(int precision)
     {
-        return Objects.hash(millis, sessionTimeZoneKey);
+        return newInstanceWithRounding(precision, epochMicros, picosOfMicros, sessionTimeZoneKey);
     }
 
     @Override
-    public boolean equals(Object obj)
+    public boolean equals(Object o)
     {
-        if (this == obj) {
+        if (this == o) {
             return true;
         }
-        if (obj == null || getClass() != obj.getClass()) {
+        if (o == null || getClass() != o.getClass()) {
             return false;
         }
-        SqlTimestamp other = (SqlTimestamp) obj;
-        return Objects.equals(this.millis, other.millis) &&
-                Objects.equals(this.sessionTimeZoneKey, other.sessionTimeZoneKey);
+        SqlTimestamp that = (SqlTimestamp) o;
+        return epochMicros == that.epochMicros &&
+                picosOfMicros == that.picosOfMicros &&
+                precision == that.precision &&
+                sessionTimeZoneKey.equals(that.sessionTimeZoneKey);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(epochMicros, picosOfMicros, precision, sessionTimeZoneKey);
     }
 
     @JsonValue
@@ -103,9 +189,38 @@ public String toString()
                 .map(ZoneId::of)
                 .orElse(ZoneOffset.UTC);
 
-        return Instant.ofEpochMilli(millis)
-                .atZone(zoneId)
-                .format(JSON_FORMATTER);
+        return formatTimestamp(precision, epochMicros, picosOfMicros, zoneId);
+    }
+
+    private static String formatTimestamp(int precision, long epochMicros, int picosOfMicro, ZoneId zoneId)
+    {
+        Instant instant = Instant.ofEpochSecond(Math.floorDiv(epochMicros, MICROSECONDS_PER_SECOND));
+        LocalDateTime dateTime = LocalDateTime.ofInstant(instant, zoneId);
+
+        StringBuilder builder = new StringBuilder();
+        builder.append(TIMESTAMP_FORMATTER.format(dateTime));
+        if (precision > 0) {
+            long picoFraction = ((long) floorMod(epochMicros, MICROSECONDS_PER_SECOND)) * PICOSECONDS_PER_MICROSECOND + picosOfMicro;
+            long scaledFraction = picoFraction / POWERS_OF_TEN[MAX_PRECISION - precision];
+            builder.append(".");
+            builder.append(format("%0" + precision + "d", scaledFraction));
+        }
+
+        return builder.toString();
+    }
+
+    private static long round(long value, int magnitude)
+    {
+        return roundDiv(value, POWERS_OF_TEN[magnitude]) * POWERS_OF_TEN[magnitude];
+    }
+
+    private static long roundDiv(long value, long factor)
+    {
+        if (value >= 0) {
+            return (value + (factor / 2)) / factor;
+        }
+
+        return (value + 1 - (factor / 2)) / factor;
     }
 
     private static void checkState(boolean condition, String message)
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/TimeZoneKey.java b/presto-spi/src/main/java/io/prestosql/spi/type/TimeZoneKey.java
index 0b0286f9dba47..6f48b47b9380c 100644
--- a/presto-spi/src/main/java/io/prestosql/spi/type/TimeZoneKey.java
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/TimeZoneKey.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.time.ZoneId;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
@@ -168,6 +169,11 @@ public String getId()
         return id;
     }
 
+    public ZoneId getZoneId()
+    {
+        return ZoneId.of(id);
+    }
+
     @JsonValue
     public short getKey()
     {
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/TimestampParametricType.java b/presto-spi/src/main/java/io/prestosql/spi/type/TimestampParametricType.java
new file mode 100644
index 0000000000000..20b6c76b3f6bd
--- /dev/null
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/TimestampParametricType.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.prestosql.spi.type;
+
+import java.util.List;
+
+public class TimestampParametricType
+        implements ParametricType
+{
+    public static final TimestampParametricType TIMESTAMP = new TimestampParametricType();
+
+    @Override
+    public String getName()
+    {
+        return StandardTypes.TIMESTAMP;
+    }
+
+    @Override
+    public Type createType(TypeManager typeManager, List parameters)
+    {
+        if (parameters.isEmpty()) {
+            return TimestampType.TIMESTAMP;
+        }
+        if (parameters.size() != 1) {
+            throw new IllegalArgumentException("Expected exactly one parameter for TIMESTAMP");
+        }
+
+        TypeParameter parameter = parameters.get(0);
+
+        if (!parameter.isLongLiteral()) {
+            throw new IllegalArgumentException("TIMESTAMP precision must be a number");
+        }
+
+        long precision = parameter.getLongLiteral();
+
+        if (precision < 0 || precision > TimestampType.MAX_PRECISION) {
+            throw new IllegalArgumentException("Invalid TIMESTAMP precision " + precision);
+        }
+
+        return TimestampType.createTimestampType((int) precision);
+    }
+}
diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/TimestampType.java b/presto-spi/src/main/java/io/prestosql/spi/type/TimestampType.java
index 90b175814f47f..7bd4ac6bfe19a 100644
--- a/presto-spi/src/main/java/io/prestosql/spi/type/TimestampType.java
+++ b/presto-spi/src/main/java/io/prestosql/spi/type/TimestampType.java
@@ -13,49 +13,63 @@
  */
 package io.prestosql.spi.type;
 
-import io.prestosql.spi.block.Block;
-import io.prestosql.spi.connector.ConnectorSession;
+import io.prestosql.spi.PrestoException;
+
+import static io.prestosql.spi.StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE;
+import static java.lang.String.format;
 
 /**
  * A timestamp is stored as milliseconds from 1970-01-01T00:00:00 UTC and is to be interpreted as date-time in UTC.
  * In legacy timestamp semantics, timestamp is stored as milliseconds from 1970-01-01T00:00:00 UTC and is to be
  * interpreted in session time zone.
  */
-public final class TimestampType
-        extends AbstractLongType
+public abstract class TimestampType
+        extends AbstractType
+        implements FixedWidthType
 {
-    public static final TimestampType TIMESTAMP = new TimestampType();
+    public static final int MAX_PRECISION = 12;
 
-    private TimestampType()
-    {
-        super(new TypeSignature(StandardTypes.TIMESTAMP));
-    }
+    public static final int MAX_SHORT_PRECISION = 6;
+    private static final int DEFAULT_PRECISION = 3; // TODO: should be 6 per SQL spec
 
-    @Override
-    public Object getObjectValue(ConnectorSession session, Block block, int position)
+    @Deprecated
+    public static final TimestampType TIMESTAMP = createTimestampType(DEFAULT_PRECISION);
+
+    private final int precision;
+
+    public static TimestampType createTimestampType(int precision)
     {
-        if (block.isNull(position)) {
-            return null;
+        if (precision < 0 || precision > MAX_PRECISION) {
+            throw new PrestoException(NUMERIC_VALUE_OUT_OF_RANGE, format("TIMESTAMP precision must be in range [0, %s]", MAX_PRECISION));
         }
 
-        if (session.isLegacyTimestamp()) {
-            return new SqlTimestamp(block.getLong(position, 0), session.getTimeZoneKey());
-        }
-        else {
-            return new SqlTimestamp(block.getLong(position, 0));
+        if (precision <= MAX_SHORT_PRECISION) {
+            return new ShortTimestampType(precision);
         }
+
+        return new LongTimestampType(precision);
+    }
+
+    protected TimestampType(int precision, Class javaType)
+    {
+        super(new TypeSignature(StandardTypes.TIMESTAMP, TypeSignatureParameter.numericParameter(precision)), javaType);
+        this.precision = precision;
+    }
+
+    public int getPrecision()
+    {
+        return precision;
     }
 
     @Override
-    @SuppressWarnings("EqualsWhichDoesntCheckParameterClass")
-    public boolean equals(Object other)
+    public boolean isComparable()
     {
-        return other == TIMESTAMP;
+        return true;
     }
 
     @Override
-    public int hashCode()
+    public boolean isOrderable()
     {
-        return getClass().hashCode();
+        return true;
     }
 }
diff --git a/presto-spi/src/test/java/io/prestosql/spi/type/TestSqlTimestamp.java b/presto-spi/src/test/java/io/prestosql/spi/type/TestSqlTimestamp.java
new file mode 100644
index 0000000000000..56cb570bd1a8b
--- /dev/null
+++ b/presto-spi/src/test/java/io/prestosql/spi/type/TestSqlTimestamp.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.prestosql.spi.type;
+
+import org.testng.annotations.Test;
+
+import static io.prestosql.spi.type.SqlTimestamp.newInstance;
+import static io.prestosql.spi.type.SqlTimestamp.newLegacyInstance;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestSqlTimestamp
+{
+    @Test
+    public void testBaseline()
+    {
+        assertThat(newInstance(0, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00");
+        assertThat(newInstance(1, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.0");
+        assertThat(newInstance(2, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.00");
+        assertThat(newInstance(3, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.000");
+        assertThat(newInstance(4, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.0000");
+        assertThat(newInstance(5, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.00000");
+        assertThat(newInstance(6, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.000000");
+        assertThat(newInstance(7, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.0000000");
+        assertThat(newInstance(8, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.00000000");
+        assertThat(newInstance(9, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.000000000");
+        assertThat(newInstance(10, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.0000000000");
+        assertThat(newInstance(11, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.00000000000");
+        assertThat(newInstance(12, 0, 0).toString()).isEqualTo("1970-01-01 00:00:00.000000000000");
+    }
+
+    @Test
+    public void testPositiveEpoch()
+    {
+        // round down
+        // represents a timestamp of 1970-01-01 00:00:00.111111111111
+        assertThat(newInstance(0, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00");
+        assertThat(newInstance(1, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.1");
+        assertThat(newInstance(2, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.11");
+        assertThat(newInstance(3, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.111");
+        assertThat(newInstance(4, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.1111");
+        assertThat(newInstance(5, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.11111");
+        assertThat(newInstance(6, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.111111");
+        assertThat(newInstance(7, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.1111111");
+        assertThat(newInstance(8, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.11111111");
+        assertThat(newInstance(9, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.111111111");
+        assertThat(newInstance(10, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.1111111111");
+        assertThat(newInstance(11, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.11111111111");
+        assertThat(newInstance(12, 111111, 111111).toString()).isEqualTo("1970-01-01 00:00:00.111111111111");
+
+        // round up
+        // represents a timestamp of 1970-01-01 00:00:00.555555555555
+        assertThat(newInstance(0, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:01");
+        assertThat(newInstance(1, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.6");
+        assertThat(newInstance(2, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.56");
+        assertThat(newInstance(3, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.556");
+        assertThat(newInstance(4, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.5556");
+        assertThat(newInstance(5, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.55556");
+        assertThat(newInstance(6, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.555556");
+        assertThat(newInstance(7, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.5555556");
+        assertThat(newInstance(8, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.55555556");
+        assertThat(newInstance(9, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.555555556");
+        assertThat(newInstance(10, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.5555555556");
+        assertThat(newInstance(11, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.55555555556");
+        assertThat(newInstance(12, 555555, 555555).toString()).isEqualTo("1970-01-01 00:00:00.555555555555");
+    }
+
+    @Test
+    public void testNegativeEpoch()
+    {
+        // round down
+        // represents a timestamp of 1969-12-31 23:59:59.111111111111
+        assertThat(newInstance(0, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59");
+        assertThat(newInstance(1, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.1");
+        assertThat(newInstance(2, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.11");
+        assertThat(newInstance(3, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.111");
+        assertThat(newInstance(4, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.1111");
+        assertThat(newInstance(5, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.11111");
+        assertThat(newInstance(6, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.111111");
+        assertThat(newInstance(7, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.1111111");
+        assertThat(newInstance(8, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.11111111");
+        assertThat(newInstance(9, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.111111111");
+        assertThat(newInstance(10, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.1111111111");
+        assertThat(newInstance(11, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.11111111111");
+        assertThat(newInstance(12, -888889, 111111).toString()).isEqualTo("1969-12-31 23:59:59.111111111111");
+
+        // round up
+        // represents a timestamp of 1969-12-31 23:59:59.555555555555
+        assertThat(newInstance(0, -444445, 555555).toString()).isEqualTo("1970-01-01 00:00:00");
+        assertThat(newInstance(1, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.6");
+        assertThat(newInstance(2, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.56");
+        assertThat(newInstance(3, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.556");
+        assertThat(newInstance(4, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.5556");
+        assertThat(newInstance(5, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.55556");
+        assertThat(newInstance(6, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.555556");
+        assertThat(newInstance(7, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.5555556");
+        assertThat(newInstance(8, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.55555556");
+        assertThat(newInstance(9, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.555555556");
+        assertThat(newInstance(10, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.5555555556");
+        assertThat(newInstance(11, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.55555555556");
+        assertThat(newInstance(12, -444445, 555555).toString()).isEqualTo("1969-12-31 23:59:59.555555555555");
+    }
+
+    @Test
+    public void testRoundTo()
+    {
+        // positive epoch, round down
+        assertThat(newInstance(12, 111111, 111111).roundTo(0).toString()).isEqualTo("1970-01-01 00:00:00");
+        assertThat(newInstance(12, 111111, 111111).roundTo(1).toString()).isEqualTo("1970-01-01 00:00:00.1");
+        assertThat(newInstance(12, 111111, 111111).roundTo(2).toString()).isEqualTo("1970-01-01 00:00:00.11");
+        assertThat(newInstance(12, 111111, 111111).roundTo(3).toString()).isEqualTo("1970-01-01 00:00:00.111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(4).toString()).isEqualTo("1970-01-01 00:00:00.1111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(5).toString()).isEqualTo("1970-01-01 00:00:00.11111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(6).toString()).isEqualTo("1970-01-01 00:00:00.111111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(7).toString()).isEqualTo("1970-01-01 00:00:00.1111111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(8).toString()).isEqualTo("1970-01-01 00:00:00.11111111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(9).toString()).isEqualTo("1970-01-01 00:00:00.111111111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(10).toString()).isEqualTo("1970-01-01 00:00:00.1111111111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(11).toString()).isEqualTo("1970-01-01 00:00:00.11111111111");
+        assertThat(newInstance(12, 111111, 111111).roundTo(12).toString()).isEqualTo("1970-01-01 00:00:00.111111111111");
+
+        // positive epoch, round up
+        assertThat(newInstance(12, 555555, 555555).roundTo(0).toString()).isEqualTo("1970-01-01 00:00:01");
+        assertThat(newInstance(12, 555555, 555555).roundTo(1).toString()).isEqualTo("1970-01-01 00:00:00.6");
+        assertThat(newInstance(12, 555555, 555555).roundTo(2).toString()).isEqualTo("1970-01-01 00:00:00.56");
+        assertThat(newInstance(12, 555555, 555555).roundTo(3).toString()).isEqualTo("1970-01-01 00:00:00.556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(4).toString()).isEqualTo("1970-01-01 00:00:00.5556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(5).toString()).isEqualTo("1970-01-01 00:00:00.55556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(6).toString()).isEqualTo("1970-01-01 00:00:00.555556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(7).toString()).isEqualTo("1970-01-01 00:00:00.5555556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(8).toString()).isEqualTo("1970-01-01 00:00:00.55555556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(9).toString()).isEqualTo("1970-01-01 00:00:00.555555556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(10).toString()).isEqualTo("1970-01-01 00:00:00.5555555556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(11).toString()).isEqualTo("1970-01-01 00:00:00.55555555556");
+        assertThat(newInstance(12, 555555, 555555).roundTo(12).toString()).isEqualTo("1970-01-01 00:00:00.555555555555");
+
+        // negative epoch, round down
+        // represents a timestamp of 1969-12-31 23:59:59.111111111111
+        assertThat(newInstance(12, -888889, 111111).roundTo(0).toString()).isEqualTo("1969-12-31 23:59:59");
+        assertThat(newInstance(12, -888889, 111111).roundTo(1).toString()).isEqualTo("1969-12-31 23:59:59.1");
+        assertThat(newInstance(12, -888889, 111111).roundTo(2).toString()).isEqualTo("1969-12-31 23:59:59.11");
+        assertThat(newInstance(12, -888889, 111111).roundTo(3).toString()).isEqualTo("1969-12-31 23:59:59.111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(4).toString()).isEqualTo("1969-12-31 23:59:59.1111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(5).toString()).isEqualTo("1969-12-31 23:59:59.11111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(6).toString()).isEqualTo("1969-12-31 23:59:59.111111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(7).toString()).isEqualTo("1969-12-31 23:59:59.1111111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(8).toString()).isEqualTo("1969-12-31 23:59:59.11111111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(9).toString()).isEqualTo("1969-12-31 23:59:59.111111111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(10).toString()).isEqualTo("1969-12-31 23:59:59.1111111111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(11).toString()).isEqualTo("1969-12-31 23:59:59.11111111111");
+        assertThat(newInstance(12, -888889, 111111).roundTo(12).toString()).isEqualTo("1969-12-31 23:59:59.111111111111");
+
+        // negative epoch, round up
+        // represents a timestamp of 1969-12-31 23:59:59.555555555555
+        assertThat(newInstance(12, -444445, 555555).roundTo(0).toString()).isEqualTo("1970-01-01 00:00:00");
+        assertThat(newInstance(12, -444445, 555555).roundTo(1).toString()).isEqualTo("1969-12-31 23:59:59.6");
+        assertThat(newInstance(12, -444445, 555555).roundTo(2).toString()).isEqualTo("1969-12-31 23:59:59.56");
+        assertThat(newInstance(12, -444445, 555555).roundTo(3).toString()).isEqualTo("1969-12-31 23:59:59.556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(4).toString()).isEqualTo("1969-12-31 23:59:59.5556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(5).toString()).isEqualTo("1969-12-31 23:59:59.55556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(6).toString()).isEqualTo("1969-12-31 23:59:59.555556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(7).toString()).isEqualTo("1969-12-31 23:59:59.5555556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(8).toString()).isEqualTo("1969-12-31 23:59:59.55555556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(9).toString()).isEqualTo("1969-12-31 23:59:59.555555556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(10).toString()).isEqualTo("1969-12-31 23:59:59.5555555556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(11).toString()).isEqualTo("1969-12-31 23:59:59.55555555556");
+        assertThat(newInstance(12, -444445, 555555).roundTo(12).toString()).isEqualTo("1969-12-31 23:59:59.555555555555");
+    }
+
+    @Test
+    public void testLegacy()
+    {
+        assertThat(newLegacyInstance(12, 0, 0, TimeZoneKey.getTimeZoneKey("America/Los_Angeles")).toString())
+                .isEqualTo("1969-12-31 16:00:00.000000000000");
+
+        // positive epoch
+        assertThat(newLegacyInstance(12, 123456, 789123, TimeZoneKey.getTimeZoneKey("America/Los_Angeles")).toString())
+                .isEqualTo("1969-12-31 16:00:00.123456789123");
+
+        // negative epoch
+        assertThat(newLegacyInstance(12, -876544, 789123, TimeZoneKey.getTimeZoneKey("America/Los_Angeles")).toString())
+                .isEqualTo("1969-12-31 15:59:59.123456789123");
+    }
+}
diff --git a/presto-teradata-functions/src/main/java/io/prestosql/teradata/functions/TeradataDateFunctions.java b/presto-teradata-functions/src/main/java/io/prestosql/teradata/functions/TeradataDateFunctions.java
index 3671abca7d49b..3326ddf3665af 100644
--- a/presto-teradata-functions/src/main/java/io/prestosql/teradata/functions/TeradataDateFunctions.java
+++ b/presto-teradata-functions/src/main/java/io/prestosql/teradata/functions/TeradataDateFunctions.java
@@ -92,7 +92,7 @@ public static long toDate(ConnectorSession session, @SqlType(StandardTypes.VARCH
 
     @Description("Converts a string to a TIMESTAMP data type")
     @ScalarFunction("to_timestamp")
-    @SqlType(StandardTypes.TIMESTAMP)
+    @SqlType("timestamp(3)")
     public static long toTimestamp(
             ConnectorSession session,
             @SqlType(StandardTypes.VARCHAR) Slice dateTime,
diff --git a/presto-teradata-functions/src/test/java/io/prestosql/teradata/functions/TestTeradataDateFunctions.java b/presto-teradata-functions/src/test/java/io/prestosql/teradata/functions/TestTeradataDateFunctions.java
index 4ece71e2357ce..01a25858690b9 100644
--- a/presto-teradata-functions/src/test/java/io/prestosql/teradata/functions/TestTeradataDateFunctions.java
+++ b/presto-teradata-functions/src/test/java/io/prestosql/teradata/functions/TestTeradataDateFunctions.java
@@ -131,7 +131,7 @@ private void assertTimestamp(String projection, int year, int month, int day, in
         assertFunction(
                 projection,
                 TimestampType.TIMESTAMP,
-                sqlTimestampOf(year, month, day, hour, minutes, seconds, 0, SESSION));
+                sqlTimestampOf(3, year, month, day, hour, minutes, seconds, 0, SESSION));
     }
 
     private void assertDate(String projection, int year, int month, int day)
diff --git a/presto-testing/src/main/java/io/prestosql/testing/H2QueryRunner.java b/presto-testing/src/main/java/io/prestosql/testing/H2QueryRunner.java
index 06cff7da1e52b..37d6b1fc4d5e4 100644
--- a/presto-testing/src/main/java/io/prestosql/testing/H2QueryRunner.java
+++ b/presto-testing/src/main/java/io/prestosql/testing/H2QueryRunner.java
@@ -25,6 +25,7 @@
 import io.prestosql.spi.type.ArrayType;
 import io.prestosql.spi.type.CharType;
 import io.prestosql.spi.type.DecimalType;
+import io.prestosql.spi.type.TimestampType;
 import io.prestosql.spi.type.Type;
 import io.prestosql.spi.type.VarcharType;
 import io.prestosql.tpch.TpchTable;
@@ -73,7 +74,6 @@
 import static io.prestosql.spi.type.SmallintType.SMALLINT;
 import static io.prestosql.spi.type.TimeType.TIME;
 import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
-import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
 import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
 import static io.prestosql.spi.type.TinyintType.TINYINT;
 import static io.prestosql.spi.type.VarbinaryType.VARBINARY;
@@ -323,7 +323,7 @@ else if (TIME.equals(type)) {
                     else if (TIME_WITH_TIME_ZONE.equals(type)) {
                         throw new UnsupportedOperationException("H2 does not support TIME WITH TIME ZONE");
                     }
-                    else if (TIMESTAMP.equals(type)) {
+                    else if (type instanceof TimestampType) {
                         // resultSet.getTimestamp(i) doesn't work if JVM's zone had forward offset at the date/time being retrieved
                         LocalDateTime timestampValue;
                         try {
diff --git a/presto-testing/src/main/java/io/prestosql/testing/TestingPrestoClient.java b/presto-testing/src/main/java/io/prestosql/testing/TestingPrestoClient.java
index 64c1845f86cb5..1e94cba810a66 100644
--- a/presto-testing/src/main/java/io/prestosql/testing/TestingPrestoClient.java
+++ b/presto-testing/src/main/java/io/prestosql/testing/TestingPrestoClient.java
@@ -27,6 +27,7 @@
 import io.prestosql.spi.type.RowType;
 import io.prestosql.spi.type.SqlTimestamp;
 import io.prestosql.spi.type.SqlTimestampWithTimeZone;
+import io.prestosql.spi.type.TimestampType;
 import io.prestosql.spi.type.Type;
 import io.prestosql.spi.type.VarcharType;
 import io.prestosql.type.SqlIntervalDayTime;
@@ -63,7 +64,6 @@
 import static io.prestosql.spi.type.SmallintType.SMALLINT;
 import static io.prestosql.spi.type.TimeType.TIME;
 import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE;
-import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
 import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE;
 import static io.prestosql.spi.type.TinyintType.TINYINT;
 import static io.prestosql.spi.type.VarbinaryType.VARBINARY;
@@ -223,7 +223,7 @@ else if (TIME_WITH_TIME_ZONE.equals(type)) {
                 return timeWithZoneOffsetFormat.parse(((String) value), OffsetTime::from);
             }
         }
-        else if (TIMESTAMP.equals(type)) {
+        else if (type instanceof TimestampType) {
             return SqlTimestamp.JSON_FORMATTER.parse((String) value, LocalDateTime::from);
         }
         else if (TIMESTAMP_WITH_TIME_ZONE.equals(type)) {
diff --git a/presto-tests/src/test/java/io/prestosql/tests/AbstractTestEngineOnlyQueries.java b/presto-tests/src/test/java/io/prestosql/tests/AbstractTestEngineOnlyQueries.java
index 8f2213168c2c9..485bbcb98ee65 100644
--- a/presto-tests/src/test/java/io/prestosql/tests/AbstractTestEngineOnlyQueries.java
+++ b/presto-tests/src/test/java/io/prestosql/tests/AbstractTestEngineOnlyQueries.java
@@ -130,7 +130,7 @@ public void testLocallyUnrepresentableTimeLiterals()
         LocalDateTime localTimeThatDidNotExist = LocalDateTime.of(2017, 4, 2, 2, 10);
         checkState(ZoneId.systemDefault().getRules().getValidOffsets(localTimeThatDidNotExist).isEmpty(), "This test assumes certain JVM time zone");
         // This tests that both Presto runner and H2 can return TIMESTAMP value that never happened in JVM's zone (e.g. is not representable using java.sql.Timestamp)
-        @Language("SQL") String sql = DateTimeFormatter.ofPattern("'SELECT TIMESTAMP '''uuuu-MM-dd HH:mm:ss''").format(localTimeThatDidNotExist);
+        @Language("SQL") String sql = DateTimeFormatter.ofPattern("'SELECT TIMESTAMP '''uuuu-MM-dd HH:mm:ss.SSS''").format(localTimeThatDidNotExist);
         assertEquals(computeScalar(sql), localTimeThatDidNotExist); // this tests Presto and the QueryRunner
         assertQuery(sql); // this tests H2QueryRunner