Skip to content

Commit

Permalink
Fix timestamp semantics
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Aug 13, 2020
1 parent ddee904 commit e9a6ff9
Show file tree
Hide file tree
Showing 275 changed files with 8,346 additions and 10,345 deletions.
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -497,7 +497,7 @@
<dependency>
<groupId>io.prestosql.hive</groupId>
<artifactId>hive-apache</artifactId>
<version>3.0.0-6</version>
<version>3.1.2-1</version>
</dependency>

<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -331,14 +331,10 @@ private static Object convert(Object value, Type type)
}

if (type.equals(TIME)) {
// TODO this likely is incorrect, passing the millis value interpreted in UTC into millis value interpreted in JVM's zone
// TODO account for non-legacy timestamp
return new Time((long) value);
}

if (type.equals(TIMESTAMP)) {
// TODO this likely is incorrect, passing the millis value interpreted in UTC into millis value interpreted in JVM's zone
// TODO account for non-legacy timestamp
return new Timestamp((long) value);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -325,7 +325,7 @@ public Optional<ColumnMapping> toPrestoType(ConnectorSession session, Connection
if (mapping.isPresent()) {
return mapping;
}
Optional<ColumnMapping> connectorMapping = jdbcTypeToPrestoType(session, typeHandle);
Optional<ColumnMapping> connectorMapping = jdbcTypeToPrestoType(typeHandle);
if (connectorMapping.isPresent()) {
return connectorMapping;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
import com.google.common.base.CharMatcher;
import com.google.common.primitives.Shorts;
import com.google.common.primitives.SignedBytes;
import io.prestosql.spi.connector.ConnectorSession;
import io.prestosql.spi.type.CharType;
import io.prestosql.spi.type.DecimalType;
import io.prestosql.spi.type.Decimals;
Expand All @@ -37,7 +36,6 @@
import java.time.Instant;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.ZoneId;
import java.util.Optional;

import static com.google.common.base.Preconditions.checkArgument;
Expand Down Expand Up @@ -351,42 +349,27 @@ public static LongWriteFunction timeWriteFunction()
* {@link #timestampColumnMapping} instead.
*/
@Deprecated
public static ColumnMapping timestampColumnMappingUsingSqlTimestamp(ConnectorSession session)
public static ColumnMapping timestampColumnMappingUsingSqlTimestamp()
{
if (session.isLegacyTimestamp()) {
ZoneId sessionZone = ZoneId.of(session.getTimeZoneKey().getId());
return ColumnMapping.longMapping(
TIMESTAMP,
(resultSet, columnIndex) -> {
Timestamp timestamp = resultSet.getTimestamp(columnIndex);
return toPrestoLegacyTimestamp(timestamp.toLocalDateTime(), sessionZone);
},
timestampWriteFunctionUsingSqlTimestamp(session));
}

return ColumnMapping.longMapping(
TIMESTAMP,
(resultSet, columnIndex) -> {
Timestamp timestamp = resultSet.getTimestamp(columnIndex);
return toPrestoTimestamp(timestamp.toLocalDateTime());
},
timestampWriteFunctionUsingSqlTimestamp(session));
timestampWriteFunctionUsingSqlTimestamp());
}

public static ColumnMapping timestampColumnMapping(ConnectorSession session)
public static ColumnMapping timestampColumnMapping()
{
return ColumnMapping.longMapping(
TIMESTAMP,
timestampReadFunction(session),
timestampWriteFunction(session));
timestampReadFunction(),
timestampWriteFunction());
}

public static LongReadFunction timestampReadFunction(ConnectorSession session)
public static LongReadFunction timestampReadFunction()
{
if (session.isLegacyTimestamp()) {
ZoneId sessionZone = ZoneId.of(session.getTimeZoneKey().getId());
return (resultSet, columnIndex) -> toPrestoLegacyTimestamp(resultSet.getObject(columnIndex, LocalDateTime.class), sessionZone);
}
return (resultSet, columnIndex) -> toPrestoTimestamp(resultSet.getObject(columnIndex, LocalDateTime.class));
}

Expand All @@ -397,49 +380,21 @@ public static LongReadFunction timestampReadFunction(ConnectorSession session)
* {@link #timestampWriteFunction} instead.
*/
@Deprecated
public static LongWriteFunction timestampWriteFunctionUsingSqlTimestamp(ConnectorSession connectorSession)
public static LongWriteFunction timestampWriteFunctionUsingSqlTimestamp()
{
if (connectorSession.isLegacyTimestamp()) {
ZoneId sessionZone = ZoneId.of(connectorSession.getTimeZoneKey().getId());
return (statement, index, value) -> statement.setTimestamp(index, Timestamp.valueOf(fromPrestoLegacyTimestamp(value, sessionZone)));
}
return (statement, index, value) -> statement.setTimestamp(index, Timestamp.valueOf(fromPrestoTimestamp(value)));
}

public static LongWriteFunction timestampWriteFunction(ConnectorSession session)
{
if (session.isLegacyTimestamp()) {
ZoneId sessionZone = ZoneId.of(session.getTimeZoneKey().getId());
return (statement, index, value) -> statement.setObject(index, fromPrestoLegacyTimestamp(value, sessionZone));
}
return (statement, index, value) -> {
statement.setObject(index, fromPrestoTimestamp(value));
};
}

/**
* @deprecated applicable in legacy timestamp semantics only
*/
@Deprecated
public static long toPrestoLegacyTimestamp(LocalDateTime localDateTime, ZoneId sessionZone)
public static LongWriteFunction timestampWriteFunction()
{
return localDateTime.atZone(sessionZone).toInstant().toEpochMilli();
return (statement, index, value) -> statement.setObject(index, fromPrestoTimestamp(value));
}

public static long toPrestoTimestamp(LocalDateTime localDateTime)
{
return localDateTime.atZone(UTC).toInstant().toEpochMilli();
}

/**
* @deprecated applicable in legacy timestamp semantics only
*/
@Deprecated
public static LocalDateTime fromPrestoLegacyTimestamp(long value, ZoneId sessionZone)
{
return Instant.ofEpochMilli(value).atZone(sessionZone).toLocalDateTime();
}

public static LocalDateTime fromPrestoTimestamp(long value)
{
return Instant.ofEpochMilli(value).atZone(UTC).toLocalDateTime();
Expand All @@ -451,7 +406,7 @@ public static LocalTime fromPrestoTime(long value)
return LocalTime.ofNanoOfDay(roundDiv(value, PICOSECONDS_PER_NANOSECOND) % NANOSECONDS_PER_DAY);
}

public static Optional<ColumnMapping> jdbcTypeToPrestoType(ConnectorSession session, JdbcTypeHandle type)
public static Optional<ColumnMapping> jdbcTypeToPrestoType(JdbcTypeHandle type)
{
int columnSize = type.getColumnSize();
switch (type.getJdbcType()) {
Expand Down Expand Up @@ -516,7 +471,7 @@ public static Optional<ColumnMapping> jdbcTypeToPrestoType(ConnectorSession sess

case Types.TIMESTAMP:
// TODO default to `timestampColumnMapping`
return Optional.of(timestampColumnMappingUsingSqlTimestamp(session));
return Optional.of(timestampColumnMappingUsingSqlTimestamp());
}
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,6 @@
import io.prestosql.spi.predicate.ValueSet;
import io.prestosql.spi.type.CharType;
import io.prestosql.spi.type.SqlTime;
import io.prestosql.spi.type.SqlTimestamp;
import io.prestosql.testing.DateTimeTestingUtils;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
Expand Down Expand Up @@ -70,18 +68,17 @@
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.TimeZoneKey.UTC_KEY;
import static io.prestosql.spi.type.TimestampType.TIMESTAMP;
import static io.prestosql.spi.type.TinyintType.TINYINT;
import static io.prestosql.spi.type.VarcharType.VARCHAR;
import static io.prestosql.testing.DateTimeTestingUtils.sqlTimeOf;
import static io.prestosql.testing.DateTimeTestingUtils.sqlTimestampOf;
import static io.prestosql.testing.TestingConnectorSession.SESSION;
import static java.lang.Float.floatToRawIntBits;
import static java.lang.String.format;
import static java.time.temporal.ChronoUnit.DAYS;
import static java.util.function.Function.identity;
import static org.assertj.core.api.Assertions.assertThat;
import static org.joda.time.DateTimeZone.UTC;
import static org.testng.Assert.assertEquals;

@Test(singleThreaded = true)
Expand Down Expand Up @@ -512,11 +509,7 @@ public void testAggregationWithFilter()

private static long toPrestoTimestamp(int year, int month, int day, int hour, int minute, int second)
{
SqlTimestamp sqlTimestamp = DateTimeTestingUtils.sqlTimestampOf(3, year, month, day, hour, minute, second, 0, UTC, UTC_KEY, SESSION);
if (SESSION.isLegacyTimestamp()) {
return sqlTimestamp.getMillisUtc();
}
return sqlTimestamp.getMillis();
return sqlTimestampOf(3, year, month, day, hour, minute, second, 0).getMillis();
}

private static Timestamp toTimestamp(int year, int month, int day, int hour, int minute, int second)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -264,8 +264,7 @@ public void testSelectAllTypes()
assertEquals(row.getField(6), 0.0);
assertEquals(row.getField(7), false);
assertEquals(row.getField(8), LocalDate.ofEpochDay(0));
// TODO should be 1970-01-01 00:00:00 after https://github.com/prestosql/presto/issues/37
assertEquals(row.getField(9), LocalDateTime.of(1969, 12, 31, 13, 0, 0));
assertEquals(row.getField(9), LocalDateTime.of(1970, 1, 1, 0, 0, 0));
assertEquals(row.getField(10), "****************".getBytes(UTF_8));
assertEquals(row.getField(11), new BigDecimal("0.00"));
assertEquals(row.getField(12), new BigDecimal("00000000000000000000.0000000000"));
Expand Down
1 change: 0 additions & 1 deletion presto-docs/src/main/sphinx/language.rst
Original file line number Diff line number Diff line change
Expand Up @@ -22,4 +22,3 @@ Presto also provides :doc:`numerous SQL functions and operators<functions>`.

language/types
language/reserved
language/timestamp
60 changes: 0 additions & 60 deletions presto-docs/src/main/sphinx/language/timestamp.rst

This file was deleted.

2 changes: 1 addition & 1 deletion presto-docs/src/main/sphinx/language/types.rst
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ JSON value type, which can be a JSON object, a JSON array, a JSON number, a JSON
Date and Time
-------------

See also :doc:`/functions/datetime` and :doc:`/language/timestamp`
See also :doc:`/functions/datetime`

``DATE``
^^^^^^^^
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
import io.prestosql.elasticsearch.client.ElasticsearchClient;
import io.prestosql.spi.Page;
import io.prestosql.spi.connector.ConnectorPageSource;
import io.prestosql.spi.connector.ConnectorSession;

import static io.prestosql.elasticsearch.ElasticsearchQueryBuilder.buildSearchQuery;
import static java.lang.Math.toIntExact;
Expand All @@ -33,18 +32,17 @@ class CountQueryPageSource
private final long readTimeNanos;
private long remaining;

public CountQueryPageSource(ElasticsearchClient client, ConnectorSession session, ElasticsearchTableHandle table, ElasticsearchSplit split)
public CountQueryPageSource(ElasticsearchClient client, ElasticsearchTableHandle table, ElasticsearchSplit split)
{
requireNonNull(client, "client is null");
requireNonNull(session, "session is null");
requireNonNull(table, "table is null");
requireNonNull(split, "split is null");

long start = System.nanoTime();
long count = client.count(
split.getIndex(),
split.getShard(),
buildSearchQuery(session, table.getConstraint().transform(ElasticsearchColumnHandle.class::cast), table.getQuery()));
buildSearchQuery(table.getConstraint().transform(ElasticsearchColumnHandle.class::cast), table.getQuery()));
readTimeNanos = System.nanoTime() - start;

if (table.getLimit().isPresent()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,12 +62,11 @@ public ConnectorPageSource createPageSource(
}

if (columns.isEmpty()) {
return new CountQueryPageSource(client, session, elasticsearchTable, elasticsearchSplit);
return new CountQueryPageSource(client, elasticsearchTable, elasticsearchSplit);
}

return new ScanQueryPageSource(
client,
session,
elasticsearchTable,
elasticsearchSplit,
columns.stream()
Expand Down

0 comments on commit e9a6ff9

Please sign in to comment.