Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -74,8 +74,6 @@
import static com.facebook.presto.plugin.jdbc.JdbcWarningCode.USE_OF_DEPRECATED_CONFIGURATION_PROPERTY;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.jdbcTypeToReadMapping;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.prestoTypeToWriteMapping;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.timestampReadMapping;
import static com.facebook.presto.plugin.jdbc.mapping.StandardColumnMappings.timestampReadMappingLegacy;
import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.google.common.base.MoreObjects.firstNonNull;
Expand Down Expand Up @@ -278,10 +276,6 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
@Override
public Optional<ReadMapping> toPrestoType(ConnectorSession session, JdbcTypeHandle typeHandle)
{
if (typeHandle.getJdbcType() == java.sql.Types.TIMESTAMP) {
boolean legacyTimestamp = session.getSqlFunctionProperties().isLegacyTimestamp();
return Optional.of(legacyTimestamp ? timestampReadMappingLegacy() : timestampReadMapping());
}
return jdbcTypeToReadMapping(typeHandle);
}

Expand Down Expand Up @@ -800,9 +794,9 @@ protected String toSqlType(Type type)
throw new PrestoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName());
}

public WriteMapping toWriteMapping(ConnectorSession session, Type type)
public WriteMapping toWriteMapping(Type type)
{
Optional<WriteMapping> writeMapping = prestoTypeToWriteMapping(session, type);
Optional<WriteMapping> writeMapping = prestoTypeToWriteMapping(type);
if (writeMapping.isPresent()) {
return writeMapping.get();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ default boolean schemaExists(ConnectorSession session, JdbcIdentity identity, St

Optional<ReadMapping> toPrestoType(ConnectorSession session, JdbcTypeHandle typeHandle);

WriteMapping toWriteMapping(ConnectorSession session, Type type);
WriteMapping toWriteMapping(Type type);

ConnectorSplitSource getSplits(ConnectorSession session, JdbcIdentity identity, JdbcTableLayoutHandle layoutHandle);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public JdbcPageSink(ConnectorSession session, JdbcOutputTableHandle handle, Jdbc

columnTypes = handle.getColumnTypes();
columnWriters = columnTypes.stream().map(type -> {
WriteFunction writeFunction = jdbcClient.toWriteMapping(session, type).getWriteFunction();
WriteFunction writeFunction = jdbcClient.toWriteMapping(type).getWriteFunction();
verify(type.getJavaType() == writeFunction.getJavaType(),
format("Presto type %s is not compatible with write function %s accepting %s", type, writeFunction, writeFunction.getJavaType()));
return writeFunction;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import com.facebook.presto.common.type.UuidType;
import com.facebook.presto.common.type.VarcharType;
import com.facebook.presto.plugin.jdbc.JdbcTypeHandle;
import com.facebook.presto.spi.ConnectorSession;
import com.google.common.base.CharMatcher;
import com.google.common.primitives.Shorts;
import com.google.common.primitives.SignedBytes;
Expand All @@ -39,9 +38,7 @@
import java.sql.Timestamp;
import java.sql.Types;
import java.time.Instant;
import java.util.Calendar;
import java.util.Optional;
import java.util.TimeZone;

import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.common.type.BooleanType.BOOLEAN;
Expand Down Expand Up @@ -94,7 +91,6 @@ public final class StandardColumnMappings
private StandardColumnMappings() {}

private static final ISOChronology UTC_CHRONOLOGY = ISOChronology.getInstanceUTC();
private static final Calendar UTC_CALENDAR = Calendar.getInstance(TimeZone.getTimeZone("UTC"));

public static ReadMapping booleanReadMapping()
{
Expand Down Expand Up @@ -264,37 +260,22 @@ public static WriteMapping timeWriteMapping()
public static ReadMapping timestampReadMapping()
{
return createLongReadMapping(TIMESTAMP, (resultSet, columnIndex) -> {
Timestamp timestamp = resultSet.getTimestamp(columnIndex, UTC_CALENDAR);
return timestamp.getTime();
});
}

@Deprecated
public static ReadMapping timestampReadMappingLegacy()
{
return createLongReadMapping(TIMESTAMP, (resultSet, columnIndex) -> {
/*
* TODO `resultSet.getTimestamp(columnIndex)` returns wrong value if JVM's zone had forward offset change and the local time
* corresponding to timestamp value being retrieved was not present (a 'gap'), this includes regular DST changes (e.g. Europe/Warsaw)
* and one-time policy changes (Asia/Kathmandu's shift by 15 minutes on January 1, 1986, 00:00:00).
* The problem can be averted by using `resultSet.getObject(columnIndex, LocalDateTime.class)` -- but this is not universally supported by JDBC drivers.
*/
Timestamp timestamp = resultSet.getTimestamp(columnIndex);
return timestamp.getTime();
});
}

public static WriteMapping timestampWriteMapping(TimestampType timestampType)
{
return createLongWriteMapping((statement, index, value) -> {
statement.setTimestamp(index, Timestamp.from(Instant.ofEpochSecond(
timestampType.getEpochSecond(value),
timestampType.getNanos(value))), UTC_CALENDAR);
});
}

@Deprecated
public static WriteMapping timestampWriteMappingLegacy(TimestampType timestampType)
{
return createLongWriteMapping((statement, index, value) -> {
statement.setTimestamp(index, Timestamp.from(Instant.ofEpochSecond(
timestampType.getEpochSecond(value),
timestampType.getNanos(value))));
});
return createLongWriteMapping((statement, index, value) -> statement.setTimestamp(index, Timestamp.from(Instant.ofEpochSecond(
timestampType.getEpochSecond(value),
timestampType.getNanos(value)))));
}
public static WriteMapping uuidWriteMapping()
{
Expand Down Expand Up @@ -377,7 +358,7 @@ public static Optional<ReadMapping> jdbcTypeToReadMapping(JdbcTypeHandle type)
return Optional.empty();
}

public static Optional<WriteMapping> prestoTypeToWriteMapping(ConnectorSession session, Type type)
public static Optional<WriteMapping> prestoTypeToWriteMapping(Type type)
{
if (type.equals(BOOLEAN)) {
return Optional.of(booleanWriteMapping());
Expand Down Expand Up @@ -413,8 +394,7 @@ else if (type instanceof DateType) {
return Optional.of(dateWriteMapping());
}
else if (type instanceof TimestampType) {
boolean legacyTimestamp = session.getSqlFunctionProperties().isLegacyTimestamp();
return Optional.of(legacyTimestamp ? timestampWriteMappingLegacy((TimestampType) type) : timestampWriteMapping((TimestampType) type));
return Optional.of(timestampWriteMapping((TimestampType) type));
}
else if (type.equals(TIME)) {
return Optional.of(timeWriteMapping());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,12 +37,9 @@
import java.sql.Timestamp;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
import java.util.Calendar;
import java.util.List;
import java.util.Locale;
import java.util.Optional;
import java.util.TimeZone;

import static com.facebook.airlift.testing.Assertions.assertContains;
import static com.facebook.presto.common.type.BigintType.BIGINT;
Expand Down Expand Up @@ -77,7 +74,6 @@
@Test(singleThreaded = true)
public class TestJdbcQueryBuilder
{
private static final Calendar UTC_CALENDAR = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
private TestingDatabase database;
private JdbcClient jdbcClient;
private ConnectorSession session;
Expand Down Expand Up @@ -352,7 +348,7 @@ public void testBuildSqlWithTimestamp()
ResultSet resultSet = preparedStatement.executeQuery()) {
ImmutableSet.Builder<Timestamp> builder = ImmutableSet.builder();
while (resultSet.next()) {
builder.add(resultSet.getTimestamp("col_6", UTC_CALENDAR));
builder.add((Timestamp) resultSet.getObject("col_6"));
}
assertEquals(builder.build(), ImmutableSet.of(
toTimestamp(2016, 6, 3, 0, 23, 37),
Expand Down Expand Up @@ -383,7 +379,7 @@ public void testEmptyBuildSql()

private static Timestamp toTimestamp(int year, int month, int day, int hour, int minute, int second)
{
return Timestamp.from(LocalDateTime.of(year, month, day, hour, minute, second).toInstant(ZoneOffset.UTC));
return Timestamp.valueOf(LocalDateTime.of(year, month, day, hour, minute, second));
}

private static long toDays(int year, int month, int day)
Expand Down
5 changes: 5 additions & 0 deletions presto-main/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -475,6 +475,11 @@
<artifactId>postgresql</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>com.github.luben</groupId>
<artifactId>zstd-jni</artifactId>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,7 @@
import com.facebook.presto.resourcemanager.ResourceManagerConfig;
import com.facebook.presto.resourcemanager.ResourceManagerInconsistentException;
import com.facebook.presto.resourcemanager.ResourceManagerResourceGroupService;
import com.facebook.presto.server.remotetask.DecompressionFilter;
import com.facebook.presto.server.remotetask.HttpLocationFactory;
import com.facebook.presto.server.remotetask.ReactorNettyHttpClientConfig;
import com.facebook.presto.server.thrift.FixedAddressSelector;
Expand Down Expand Up @@ -437,6 +438,10 @@ else if (serverConfig.isCoordinator()) {
// task execution
jaxrsBinder(binder).bind(TaskResource.class);
jaxrsBinder(binder).bind(ThriftTaskUpdateRequestBodyReader.class);
install(installModuleIf(
ReactorNettyHttpClientConfig.class,
config -> config.isHttp2CompressionEnabled() && config.isReactorNettyHttpClientEnabled(),
moduleBinder -> jaxrsBinder(moduleBinder).bind(DecompressionFilter.class)));

newExporter(binder).export(TaskResource.class).withGeneratedName();
jaxrsBinder(binder).bind(TaskExecutorResource.class);
Expand Down
Original file line number Diff line number Diff line change
@@ -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 com.facebook.presto.server.remotetask;

import com.facebook.airlift.log.Logger;
import com.facebook.presto.spi.PrestoException;
import com.github.luben.zstd.ZstdInputStream;
import jakarta.annotation.Priority;
import jakarta.ws.rs.Priorities;
import jakarta.ws.rs.container.ContainerRequestContext;
import jakarta.ws.rs.container.ContainerRequestFilter;
import jakarta.ws.rs.ext.Provider;

import java.io.IOException;
import java.io.InputStream;

import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.lang.String.format;

@Provider
@Priority(Priorities.ENTITY_CODER)
public class DecompressionFilter
implements ContainerRequestFilter
{
private static final Logger log = Logger.get(DecompressionFilter.class);

@Override
public void filter(ContainerRequestContext containerRequestContext)
throws IOException
{
String contentEncoding = containerRequestContext.getHeaderString("Content-Encoding");

if (contentEncoding != null && !contentEncoding.equalsIgnoreCase("identity")) {
InputStream originalStream = containerRequestContext.getEntityStream();
InputStream decompressedStream;

if (contentEncoding.equalsIgnoreCase("zstd")) {
decompressedStream = new ZstdInputStream(originalStream);
log.info("DecompressionFilter ZSTD stream");
}
else {
throw new PrestoException(NOT_SUPPORTED, format("Unsupported Content-Encoding: '%s'. Only zstd compression is supported.", contentEncoding));
}

containerRequestContext.setEntityStream(decompressedStream);
containerRequestContext.getHeaders().remove("Content-Encoding");
}
}
}
Loading
Loading