Skip to content
Closed
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 @@ -19,6 +19,8 @@
import com.facebook.presto.common.type.DecimalType;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.VarcharType;
import com.facebook.presto.plugin.jdbc.optimization.JdbcQueryGeneratorContext;
import com.facebook.presto.plugin.jdbc.optimization.JdbcSortItem;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.ConnectorSession;
Expand Down Expand Up @@ -53,6 +55,8 @@
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.function.BiFunction;
import java.util.function.Function;

import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.common.type.BooleanType.BOOLEAN;
Expand Down Expand Up @@ -136,6 +140,66 @@ public void destroy()
connectionFactory.close();
}

@Override
public boolean supportsLimit()
{
return limitFunction().isPresent();
}

protected Optional<BiFunction<String, Long, String>> limitFunction()
{
return Optional.empty();
}

private Function<String, String> applyLimit(long limit)
{
return query -> limitFunction()
.orElseThrow(() -> new PrestoException(JDBC_ERROR, "limitFunction is not set!"))
.apply(query, limit);
}

@Override
public boolean supportsTopN(List<JdbcSortItem> sortItems)
{
if (!topNFunction().isPresent()) {
return false;
}
throw new UnsupportedOperationException("topNFunction() implemented without implementing supportsTopN()");
}

protected Optional<TopNFunction> topNFunction()
{
return Optional.empty();
}

private Function<String, String> applyTopN(List<JdbcSortItem> sortItems, long limit)
{
return query -> topNFunction()
.orElseThrow(() -> new PrestoException(JDBC_ERROR, "topNFunction is not set!"))
.apply(query, sortItems, limit);
}

@FunctionalInterface
public interface TopNFunction
{
String apply(String query, List<JdbcSortItem> sortItems, long limit);
}

@Override
public String applyQueryTransformations(String query, JdbcQueryGeneratorContext context)
{
if (context.getLimit().isPresent()) {
if (context.getSortOrder().isPresent()) {
return applyTopN(context.getSortOrder().get(), context.getLimit().getAsLong()).apply(query);
}
else {
return applyLimit(context.getLimit().getAsLong()).apply(query);
}
}

return query;
}

@Override
public String getIdentifierQuote()
{
Expand Down Expand Up @@ -208,7 +272,8 @@ public JdbcTableHandle getTableHandle(JdbcIdentity identity, SchemaTableName sch
schemaTableName,
resultSet.getString("TABLE_CAT"),
resultSet.getString("TABLE_SCHEM"),
resultSet.getString("TABLE_NAME")));
resultSet.getString("TABLE_NAME"),
Optional.empty()));
}
if (tableHandles.isEmpty()) {
return null;
Expand All @@ -233,6 +298,7 @@ public List<JdbcColumnHandle> getColumns(ConnectorSession session, JdbcTableHand
while (resultSet.next()) {
JdbcTypeHandle typeHandle = new JdbcTypeHandle(
resultSet.getInt("DATA_TYPE"),
Optional.ofNullable(resultSet.getString("TYPE_NAME")),
resultSet.getInt("COLUMN_SIZE"),
resultSet.getInt("DECIMAL_DIGITS"));
Optional<ReadMapping> columnMapping = toPrestoType(session, typeHandle);
Expand Down Expand Up @@ -271,7 +337,7 @@ public ConnectorSplitSource getSplits(JdbcIdentity identity, JdbcTableLayoutHand
tableHandle.getSchemaName(),
tableHandle.getTableName(),
layoutHandle.getTupleDomain(),
layoutHandle.getAdditionalPredicate());
tableHandle.getContext());
return new FixedSplitSource(ImmutableList.of(jdbcSplit));
}

Expand Down Expand Up @@ -303,7 +369,7 @@ public PreparedStatement buildSql(ConnectorSession session, Connection connectio
split.getTableName(),
columnHandles,
split.getTupleDomain(),
split.getAdditionalPredicate());
split.getContext());
}

@Override
Expand Down Expand Up @@ -551,7 +617,8 @@ public void rollbackCreateTable(JdbcIdentity identity, JdbcOutputTableHandle han
new SchemaTableName(handle.getSchemaName(), handle.getTemporaryTableName()),
handle.getCatalogName(),
handle.getSchemaName(),
handle.getTemporaryTableName()));
handle.getTemporaryTableName(),
Optional.empty()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@
package com.facebook.presto.plugin.jdbc;

import com.facebook.presto.common.predicate.TupleDomain;
import com.facebook.presto.plugin.jdbc.optimization.JdbcQueryGeneratorContext;
import com.facebook.presto.plugin.jdbc.optimization.JdbcSortItem;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.ConnectorSession;
Expand All @@ -38,6 +40,12 @@ default boolean schemaExists(JdbcIdentity identity, String schema)
return getSchemaNames(identity).contains(schema);
}

boolean supportsLimit();

boolean supportsTopN(List<JdbcSortItem> sortItems);

String applyQueryTransformations(String query, JdbcQueryGeneratorContext context);

String getIdentifierQuote();

Set<String> getSchemaNames(JdbcIdentity identity);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.facebook.airlift.bootstrap.LifeCycleManager;
import com.facebook.airlift.log.Logger;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.plugin.jdbc.optimization.JdbcPlanOptimizerProvider;
import com.facebook.presto.spi.connector.Connector;
import com.facebook.presto.spi.connector.ConnectorAccessControl;
Expand Down Expand Up @@ -63,6 +64,7 @@ public class JdbcConnector
private final FunctionMetadataManager functionManager;
private final StandardFunctionResolution functionResolution;
private final RowExpressionService rowExpressionService;
private final TypeManager typeManager;
private final JdbcClient jdbcClient;

@Inject
Expand All @@ -77,6 +79,7 @@ public JdbcConnector(
FunctionMetadataManager functionManager,
StandardFunctionResolution functionResolution,
RowExpressionService rowExpressionService,
TypeManager typeManager,
JdbcClient jdbcClient)
{
this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null");
Expand All @@ -89,6 +92,7 @@ public JdbcConnector(
this.functionManager = requireNonNull(functionManager, "functionManager is null");
this.functionResolution = requireNonNull(functionResolution, "functionResolution is null");
this.rowExpressionService = requireNonNull(rowExpressionService, "rowExpressionService is null");
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null");
}

Expand All @@ -97,6 +101,7 @@ public ConnectorPlanOptimizerProvider getConnectorPlanOptimizerProvider()
{
return new JdbcPlanOptimizerProvider(
jdbcClient,
typeManager,
functionManager,
functionResolution,
rowExpressionService.getDeterminismEvaluator(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package com.facebook.presto.plugin.jdbc;

import com.facebook.airlift.bootstrap.Bootstrap;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.spi.ConnectorHandleResolver;
import com.facebook.presto.spi.classloader.ThreadContextClassLoader;
import com.facebook.presto.spi.connector.Connector;
Expand Down Expand Up @@ -67,6 +68,7 @@ public Connector create(String catalogName, Map<String, String> requiredConfig,
try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) {
Bootstrap app = new Bootstrap(
binder -> {
binder.bind(TypeManager.class).toInstance(context.getTypeManager());
binder.bind(FunctionMetadataManager.class).toInstance(context.getFunctionMetadataManager());
binder.bind(StandardFunctionResolution.class).toInstance(context.getStandardFunctionResolution());
binder.bind(RowExpressionService.class).toInstance(context.getRowExpressionService());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ public JdbcTableHandle getTableHandle(ConnectorSession session, SchemaTableName
public List<ConnectorTableLayoutResult> getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint<ColumnHandle> constraint, Optional<Set<ColumnHandle>> desiredColumns)
{
JdbcTableHandle tableHandle = (JdbcTableHandle) table;
ConnectorTableLayout layout = new ConnectorTableLayout(new JdbcTableLayoutHandle(tableHandle, constraint.getSummary(), Optional.empty()));
ConnectorTableLayout layout = new ConnectorTableLayout(new JdbcTableLayoutHandle(tableHandle, constraint.getSummary()));
return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary()));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
package com.facebook.presto.plugin.jdbc;

import com.facebook.presto.common.predicate.TupleDomain;
import com.facebook.presto.plugin.jdbc.optimization.JdbcExpression;
import com.facebook.presto.plugin.jdbc.optimization.JdbcQueryGeneratorContext;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorSplit;
import com.facebook.presto.spi.HostAddress;
Expand All @@ -39,7 +39,7 @@ public class JdbcSplit
private final String schemaName;
private final String tableName;
private final TupleDomain<ColumnHandle> tupleDomain;
private final Optional<JdbcExpression> additionalPredicate;
private final Optional<JdbcQueryGeneratorContext> context;

@JsonCreator
public JdbcSplit(
Expand All @@ -48,14 +48,14 @@ public JdbcSplit(
@JsonProperty("schemaName") @Nullable String schemaName,
@JsonProperty("tableName") String tableName,
@JsonProperty("tupleDomain") TupleDomain<ColumnHandle> tupleDomain,
@JsonProperty("additionalProperty") Optional<JdbcExpression> additionalPredicate)
@JsonProperty("context") Optional<JdbcQueryGeneratorContext> context)
{
this.connectorId = requireNonNull(connectorId, "connector id is null");
this.catalogName = catalogName;
this.schemaName = schemaName;
this.tableName = requireNonNull(tableName, "table name is null");
this.tupleDomain = requireNonNull(tupleDomain, "tupleDomain is null");
this.additionalPredicate = requireNonNull(additionalPredicate, "additionalPredicate is null");
this.context = requireNonNull(context, "context is null");
}

@JsonProperty
Expand Down Expand Up @@ -91,9 +91,9 @@ public TupleDomain<ColumnHandle> getTupleDomain()
}

@JsonProperty
public Optional<JdbcExpression> getAdditionalPredicate()
public Optional<JdbcQueryGeneratorContext> getContext()
{
return additionalPredicate;
return context;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.plugin.jdbc;

import com.facebook.presto.plugin.jdbc.optimization.JdbcQueryGeneratorContext;
import com.facebook.presto.spi.ConnectorTableHandle;
import com.facebook.presto.spi.SchemaTableName;
import com.fasterxml.jackson.annotation.JsonCreator;
Expand All @@ -22,6 +23,7 @@
import javax.annotation.Nullable;

import java.util.Objects;
import java.util.Optional;

import static java.util.Objects.requireNonNull;

Expand All @@ -35,20 +37,23 @@ public final class JdbcTableHandle
private final String catalogName;
private final String schemaName;
private final String tableName;
private final Optional<JdbcQueryGeneratorContext> context;

@JsonCreator
public JdbcTableHandle(
@JsonProperty("connectorId") String connectorId,
@JsonProperty("schemaTableName") SchemaTableName schemaTableName,
@JsonProperty("catalogName") @Nullable String catalogName,
@JsonProperty("schemaName") @Nullable String schemaName,
@JsonProperty("tableName") String tableName)
@JsonProperty("tableName") String tableName,
@JsonProperty("context") Optional<JdbcQueryGeneratorContext> context)
{
this.connectorId = requireNonNull(connectorId, "connectorId is null");
this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null");
this.catalogName = catalogName;
this.schemaName = schemaName;
this.tableName = requireNonNull(tableName, "tableName is null");
this.context = requireNonNull(context, "context is null");
}

@JsonProperty
Expand Down Expand Up @@ -83,6 +88,12 @@ public String getTableName()
return tableName;
}

@JsonProperty
public Optional<JdbcQueryGeneratorContext> getContext()
{
return context;
}

@Override
public boolean equals(Object obj)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,12 @@
package com.facebook.presto.plugin.jdbc;

import com.facebook.presto.common.predicate.TupleDomain;
import com.facebook.presto.plugin.jdbc.optimization.JdbcExpression;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorTableLayoutHandle;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;

import java.util.Objects;
import java.util.Optional;

import static java.util.Objects.requireNonNull;

Expand All @@ -30,23 +28,14 @@ public class JdbcTableLayoutHandle
{
private final JdbcTableHandle table;
private final TupleDomain<ColumnHandle> tupleDomain;
private final Optional<JdbcExpression> additionalPredicate;

@JsonCreator
public JdbcTableLayoutHandle(
@JsonProperty("table") JdbcTableHandle table,
@JsonProperty("tupleDomain") TupleDomain<ColumnHandle> domain,
@JsonProperty("additionalPredicate") Optional<JdbcExpression> additionalPredicate)
@JsonProperty("tupleDomain") TupleDomain<ColumnHandle> domain)
{
this.table = requireNonNull(table, "table is null");
this.tupleDomain = requireNonNull(domain, "tupleDomain is null");
this.additionalPredicate = additionalPredicate;
}

@JsonProperty
public Optional<JdbcExpression> getAdditionalPredicate()
{
return additionalPredicate;
}

@JsonProperty
Expand All @@ -72,14 +61,13 @@ public boolean equals(Object o)
}
JdbcTableLayoutHandle that = (JdbcTableLayoutHandle) o;
return Objects.equals(table, that.table) &&
Objects.equals(tupleDomain, that.tupleDomain) &&
Objects.equals(additionalPredicate, that.additionalPredicate);
Objects.equals(tupleDomain, that.tupleDomain);
}

@Override
public int hashCode()
{
return Objects.hash(table, tupleDomain, additionalPredicate);
return Objects.hash(table, tupleDomain);
}

@Override
Expand Down
Loading