Skip to content

Commit

Permalink
Provide backward-compatibility mode for clients
Browse files Browse the repository at this point in the history
If clients declare they support the PARAMETRIC_DATETIME capability,
datetime types will be rendered with variable precision.
  • Loading branch information
martint committed Jun 6, 2020
1 parent 21e3ddf commit 9905ef8
Show file tree
Hide file tree
Showing 3 changed files with 40 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,5 +15,13 @@

public enum ClientCapabilities
{
PATH;
PATH,
// Whether clients support datetime types with variable precision
// timestamp(p) with time zone
// timestamp(p) without time zone
// time(p) with time zone
// time(p) without time zone
// interval X(p1) to Y(p2)
// When this capability is not set, the server returns datetime types with precision = 3
PARAMETRIC_DATETIME;
}
19 changes: 13 additions & 6 deletions presto-main/src/main/java/io/prestosql/server/protocol/Query.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import io.prestosql.Session;
import io.prestosql.client.ClientCapabilities;
import io.prestosql.client.ClientTypeSignature;
import io.prestosql.client.ClientTypeSignatureParameter;
import io.prestosql.client.Column;
Expand Down Expand Up @@ -91,6 +92,7 @@
import static io.prestosql.execution.QueryState.FAILED;
import static io.prestosql.server.protocol.Slug.Context.EXECUTING_QUERY;
import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static io.prestosql.spi.type.StandardTypes.TIMESTAMP;
import static io.prestosql.util.Failures.toFailure;
import static io.prestosql.util.MoreLists.mappedCopy;
import static java.lang.String.format;
Expand All @@ -113,6 +115,7 @@ class Query
private final ScheduledExecutorService timeoutExecutor;

private final PagesSerde serde;
private final boolean supportsParametricDateTime;

@GuardedBy("this")
private OptionalLong nextToken = OptionalLong.of(0);
Expand Down Expand Up @@ -210,7 +213,7 @@ private Query(
this.exchangeClient = exchangeClient;
this.resultsProcessorExecutor = resultsProcessorExecutor;
this.timeoutExecutor = timeoutExecutor;

this.supportsParametricDateTime = session.getClientCapabilities().contains(ClientCapabilities.PARAMETRIC_DATETIME.toString());
serde = new PagesSerdeFactory(blockEncodingSerde, isExchangeCompressionEnabled(session)).createPagesSerde();
}

Expand Down Expand Up @@ -405,7 +408,7 @@ private synchronized QueryResults getNextResult(long token, UriInfo uriInfo, Dat
Page page = serde.deserialize(serializedPage);
bytes += page.getLogicalSizeInBytes();
rows += page.getPositionCount();
pages.add(new RowIterable(session.toConnectorSession(), types, page));
pages.add(new RowIterable(session, types, page));
}
if (rows > 0) {
// client implementations do not properly handle empty list of data
Expand Down Expand Up @@ -570,20 +573,24 @@ private URI createPartialCancelUri(int stage, UriInfo uriInfo, long nextToken)
.build();
}

private static Column createColumn(String name, Type type)
private Column createColumn(String name, Type type)
{
TypeSignature signature = type.getTypeSignature();
return new Column(name, type.getDisplayName(), toClientTypeSignature(signature));
}

private static ClientTypeSignature toClientTypeSignature(TypeSignature signature)
private ClientTypeSignature toClientTypeSignature(TypeSignature signature)
{
if (signature.getBase().equalsIgnoreCase(TIMESTAMP) && !supportsParametricDateTime) {
return new ClientTypeSignature(TIMESTAMP);
}

return new ClientTypeSignature(signature.getBase(), signature.getParameters().stream()
.map(Query::toClientTypeSignatureParameter)
.map(this::toClientTypeSignatureParameter)
.collect(toImmutableList()));
}

private static ClientTypeSignatureParameter toClientTypeSignatureParameter(TypeSignatureParameter parameter)
private ClientTypeSignatureParameter toClientTypeSignatureParameter(TypeSignatureParameter parameter)
{
switch (parameter.getKind()) {
case TYPE:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,13 @@

import com.google.common.collect.AbstractIterator;
import com.google.common.collect.ImmutableList;
import io.prestosql.Session;
import io.prestosql.client.ClientCapabilities;
import io.prestosql.spi.Page;
import io.prestosql.spi.block.Block;
import io.prestosql.spi.connector.ConnectorSession;
import io.prestosql.spi.type.SqlTimestamp;
import io.prestosql.spi.type.TimestampType;
import io.prestosql.spi.type.Type;

import java.util.ArrayList;
Expand All @@ -30,11 +34,11 @@
class RowIterable
implements Iterable<List<Object>>
{
private final ConnectorSession session;
private final Session session;
private final List<Type> types;
private final Page page;

public RowIterable(ConnectorSession session, List<Type> types, Page page)
public RowIterable(Session session, List<Type> types, Page page)
{
this.session = session;
this.types = ImmutableList.copyOf(requireNonNull(types, "types is null"));
Expand All @@ -51,13 +55,15 @@ private static class RowIterator
extends AbstractIterator<List<Object>>
{
private final ConnectorSession session;
private final boolean supportsParametricDateTime;
private final List<Type> types;
private final Page page;
private int position = -1;

private RowIterator(ConnectorSession session, List<Type> types, Page page)
private RowIterator(Session session, List<Type> types, Page page)
{
this.session = session;
this.session = session.toConnectorSession();
this.supportsParametricDateTime = session.getClientCapabilities().contains(ClientCapabilities.PARAMETRIC_DATETIME.toString());
this.types = types;
this.page = page;
}
Expand All @@ -74,7 +80,14 @@ protected List<Object> computeNext()
for (int channel = 0; channel < page.getChannelCount(); channel++) {
Type type = types.get(channel);
Block block = page.getBlock(channel);
values.add(type.getObjectValue(session, block, position));

Object value = type.getObjectValue(session, block, position);

if (value != null && type instanceof TimestampType && !supportsParametricDateTime) {
value = ((SqlTimestamp) value).roundTo(3);
}

values.add(value);
}
return Collections.unmodifiableList(values);
}
Expand Down

0 comments on commit 9905ef8

Please sign in to comment.