Skip to content

Commit

Permalink
Switch to timestamp with time zone in system.runtime.*
Browse files Browse the repository at this point in the history
  • Loading branch information
findepi committed Oct 9, 2020
1 parent 8f0e6ac commit 4de8430
Show file tree
Hide file tree
Showing 5 changed files with 64 additions and 76 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -47,9 +47,10 @@
import static io.prestosql.security.AccessControlUtil.filterQueries;
import static io.prestosql.spi.connector.SystemTable.Distribution.ALL_COORDINATORS;
import static io.prestosql.spi.type.BigintType.BIGINT;
import static io.prestosql.spi.type.TimestampType.TIMESTAMP_MILLIS;
import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static io.prestosql.spi.type.TimeZoneKey.UTC_KEY;
import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS;
import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType;
import static io.prestosql.type.DateTimes.MICROSECONDS_PER_MILLISECOND;
import static java.util.Objects.requireNonNull;

public class QuerySystemTable
Expand All @@ -69,10 +70,10 @@ public class QuerySystemTable
.column("analysis_time_ms", BIGINT)
.column("planning_time_ms", BIGINT)

.column("created", TIMESTAMP_MILLIS)
.column("started", TIMESTAMP_MILLIS)
.column("last_heartbeat", TIMESTAMP_MILLIS)
.column("end", TIMESTAMP_MILLIS)
.column("created", TIMESTAMP_TZ_MILLIS)
.column("started", TIMESTAMP_TZ_MILLIS)
.column("last_heartbeat", TIMESTAMP_TZ_MILLIS)
.column("end", TIMESTAMP_TZ_MILLIS)

.column("error_type", createUnboundedVarcharType())
.column("error_code", createUnboundedVarcharType())
Expand Down Expand Up @@ -127,10 +128,10 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect
toMillis(queryStats.getAnalysisTime()),
toMillis(queryStats.getPlanningTime()),

toTimeStamp(queryStats.getCreateTime()),
toTimeStamp(queryStats.getExecutionStartTime()),
toTimeStamp(queryStats.getLastHeartbeat()),
toTimeStamp(queryStats.getEndTime()),
toTimestampWithTimeZoneMillis(queryStats.getCreateTime()),
toTimestampWithTimeZoneMillis(queryStats.getExecutionStartTime()),
toTimestampWithTimeZoneMillis(queryStats.getLastHeartbeat()),
toTimestampWithTimeZoneMillis(queryStats.getEndTime()),

Optional.ofNullable(queryInfo.getErrorType()).map(Enum::name).orElse(null),
Optional.ofNullable(queryInfo.getErrorCode()).map(ErrorCode::getName).orElse(null));
Expand All @@ -157,12 +158,12 @@ private static Long toMillis(Duration duration)
return duration.toMillis();
}

// TODO (https://github.com/prestosql/presto/issues/5464) map to timestamp with time zone
private static Long toTimeStamp(DateTime dateTime)
private static Long toTimestampWithTimeZoneMillis(DateTime dateTime)
{
if (dateTime == null) {
return null;
}
return dateTime.getMillis() * MICROSECONDS_PER_MILLISECOND;
// dateTime.getZone() is the server zone, should be of no interest to the user
return packDateTimeWithZone(dateTime.getMillis(), UTC_KEY);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,10 @@
import static io.prestosql.metadata.MetadataUtil.TableMetadataBuilder.tableMetadataBuilder;
import static io.prestosql.spi.connector.SystemTable.Distribution.ALL_NODES;
import static io.prestosql.spi.type.BigintType.BIGINT;
import static io.prestosql.spi.type.TimestampType.TIMESTAMP_MILLIS;
import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static io.prestosql.spi.type.TimeZoneKey.UTC_KEY;
import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS;
import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType;
import static io.prestosql.type.DateTimes.MICROSECONDS_PER_MILLISECOND;

public class TaskSystemTable
implements SystemTable
Expand Down Expand Up @@ -74,10 +75,10 @@ public class TaskSystemTable
.column("physical_input_bytes", BIGINT)
.column("physical_written_bytes", BIGINT)

.column("created", TIMESTAMP_MILLIS)
.column("start", TIMESTAMP_MILLIS)
.column("last_heartbeat", TIMESTAMP_MILLIS)
.column("end", TIMESTAMP_MILLIS)
.column("created", TIMESTAMP_TZ_MILLIS)
.column("start", TIMESTAMP_TZ_MILLIS)
.column("last_heartbeat", TIMESTAMP_TZ_MILLIS)
.column("end", TIMESTAMP_TZ_MILLIS)
.build();

private final TaskManager taskManager;
Expand Down Expand Up @@ -138,10 +139,10 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect
toBytes(stats.getPhysicalInputDataSize()),
toBytes(stats.getPhysicalWrittenDataSize()),

toTimeStamp(stats.getCreateTime()),
toTimeStamp(stats.getFirstStartTime()),
toTimeStamp(taskInfo.getLastHeartbeat()),
toTimeStamp(stats.getEndTime()));
toTimestampWithTimeZoneMillis(stats.getCreateTime()),
toTimestampWithTimeZoneMillis(stats.getFirstStartTime()),
toTimestampWithTimeZoneMillis(taskInfo.getLastHeartbeat()),
toTimestampWithTimeZoneMillis(stats.getEndTime()));
}
return table.build().cursor();
}
Expand All @@ -162,11 +163,12 @@ private static Long toBytes(DataSize dataSize)
return dataSize.toBytes();
}

private static Long toTimeStamp(DateTime dateTime)
private static Long toTimestampWithTimeZoneMillis(DateTime dateTime)
{
if (dateTime == null) {
return null;
}
return dateTime.getMillis() * MICROSECONDS_PER_MILLISECOND;
// dateTime.getZone() is the server zone, should be of no interest to the user
return packDateTimeWithZone(dateTime.getMillis(), UTC_KEY);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import io.prestosql.spi.type.VarcharType;
import io.prestosql.transaction.TransactionInfo;
import io.prestosql.transaction.TransactionManager;
import org.joda.time.DateTime;

import javax.inject.Inject;

Expand All @@ -41,10 +42,11 @@
import static io.prestosql.spi.connector.SystemTable.Distribution.SINGLE_COORDINATOR;
import static io.prestosql.spi.type.BigintType.BIGINT;
import static io.prestosql.spi.type.BooleanType.BOOLEAN;
import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone;
import static io.prestosql.spi.type.StandardTypes.ARRAY;
import static io.prestosql.spi.type.TimestampType.TIMESTAMP_MILLIS;
import static io.prestosql.spi.type.TimeZoneKey.UTC_KEY;
import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS;
import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType;
import static io.prestosql.type.DateTimes.MICROSECONDS_PER_MILLISECOND;
import static java.util.Objects.requireNonNull;

public class TransactionsSystemTable
Expand All @@ -63,7 +65,7 @@ public TransactionsSystemTable(Metadata metadata, TransactionManager transaction
.column("isolation_level", createUnboundedVarcharType())
.column("read_only", BOOLEAN)
.column("auto_commit_context", BOOLEAN)
.column("create_time", TIMESTAMP_MILLIS)
.column("create_time", TIMESTAMP_TZ_MILLIS)
.column("idle_time_secs", BIGINT)
.column("written_catalog", createUnboundedVarcharType())
.column("catalogs", metadata.getParameterizedType(ARRAY, ImmutableList.of(TypeSignatureParameter.typeParameter(createUnboundedVarcharType().getTypeSignature()))))
Expand Down Expand Up @@ -93,7 +95,7 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect
info.getIsolationLevel().toString(),
info.isReadOnly(),
info.isAutoCommitContext(),
info.getCreateTime().getMillis() * MICROSECONDS_PER_MILLISECOND,
toTimestampWithTimeZoneMillis(info.getCreateTime()),
(long) info.getIdleTime().getValue(TimeUnit.SECONDS),
info.getWrittenConnectorId().map(CatalogName::getCatalogName).orElse(null),
createStringsBlock(info.getCatalogNames()));
Expand All @@ -115,4 +117,10 @@ private static Block createStringsBlock(List<CatalogName> values)
}
return builder.build();
}

private static Long toTimestampWithTimeZoneMillis(DateTime dateTime)
{
// dateTime.getZone() is the server zone, should be of no interest to the user
return packDateTimeWithZone(dateTime.getMillis(), UTC_KEY);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -85,10 +85,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(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| created| timestamp(3) with time zone| YES| null| null|
system| runtime| queries| started| timestamp(3) with time zone| YES| null| null|
system| runtime| queries| last_heartbeat| timestamp(3) with time zone| YES| null| null|
system| runtime| queries| end| timestamp(3) with time zone| 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|
Expand All @@ -111,15 +111,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(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| tasks| created| timestamp(3) with time zone| YES| null| null|
system| runtime| tasks| start| timestamp(3) with time zone| YES| null| null|
system| runtime| tasks| last_heartbeat| timestamp(3) with time zone| YES| null| null|
system| runtime| tasks| end| timestamp(3) with time zone| 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(3)| YES| null| null|
system| runtime| transactions| create_time| timestamp(3) with time zone| 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|
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

import java.time.LocalDateTime;
import java.time.ZonedDateTime;
import java.util.List;
import java.util.Optional;
Expand All @@ -50,7 +49,6 @@
import static io.prestosql.spi.type.VarcharType.VARCHAR;
import static io.prestosql.testing.TestingSession.testSessionBuilder;
import static java.lang.String.format;
import static java.time.ZoneOffset.UTC;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.assertj.core.api.Assertions.assertThat;
import static org.testng.Assert.assertFalse;
Expand Down Expand Up @@ -121,19 +119,12 @@ public void testRuntimeQueriesTimestamps()
List<Object> fields = row.getFields();
assertThat(fields).hasSize(4);
for (int i = 0; i < fields.size(); i++) {
final int fieldNumber = i;
Object value = fields.get(fieldNumber);
assertThat(value)
.as("value for field " + fieldNumber)
Object value = fields.get(i);
assertThat((ZonedDateTime) value)
.as("value for field " + i)
.isNotNull()
.isInstanceOfSatisfying(LocalDateTime.class, localDatetime -> {
// TODO (https://github.com/prestosql/presto/issues/5464) map to `timestamp with time zone`
ZonedDateTime zonedDateTime = localDatetime.atZone(UTC);
assertThat(zonedDateTime)
.as("value for field " + fieldNumber)
.isAfterOrEqualTo(timeBefore)
.isBeforeOrEqualTo(timeAfter);
});
.isAfterOrEqualTo(timeBefore)
.isBeforeOrEqualTo(timeAfter);
}
}

Expand All @@ -151,19 +142,12 @@ public void testRuntimeTasksTimestamps()
List<Object> fields = row.getFields();
assertThat(fields).hasSize(4);
for (int i = 0; i < fields.size(); i++) {
final int fieldNumber = i;
Object value = fields.get(fieldNumber);
assertThat(value)
.as("value for field " + fieldNumber)
Object value = fields.get(i);
assertThat((ZonedDateTime) value)
.as("value for field " + i)
.isNotNull()
.isInstanceOfSatisfying(LocalDateTime.class, localDatetime -> {
// TODO (https://github.com/prestosql/presto/issues/5464) map to `timestamp with time zone`
ZonedDateTime zonedDateTime = localDatetime.atZone(UTC);
assertThat(zonedDateTime)
.as("value for field " + fieldNumber)
.isAfterOrEqualTo(timeBefore)
.isBeforeOrEqualTo(timeAfter);
});
.isAfterOrEqualTo(timeBefore)
.isBeforeOrEqualTo(timeAfter);
}
}

Expand All @@ -181,19 +165,12 @@ public void testRuntimeTransactionsTimestamps()
List<Object> fields = row.getFields();
assertThat(fields).hasSize(1);
for (int i = 0; i < fields.size(); i++) {
final int fieldNumber = i;
Object value = fields.get(fieldNumber);
assertThat(value)
.as("value for field " + fieldNumber)
Object value = fields.get(i);
assertThat((ZonedDateTime) value)
.as("value for field " + i)
.isNotNull()
.isInstanceOfSatisfying(LocalDateTime.class, localDatetime -> {
// TODO (https://github.com/prestosql/presto/issues/5464) map to `timestamp with time zone`
ZonedDateTime zonedDateTime = localDatetime.atZone(UTC);
assertThat(zonedDateTime)
.as("value for field " + fieldNumber)
.isAfterOrEqualTo(timeBefore)
.isBeforeOrEqualTo(timeAfter);
});
.isAfterOrEqualTo(timeBefore)
.isBeforeOrEqualTo(timeAfter);
}
}

Expand Down

0 comments on commit 4de8430

Please sign in to comment.