Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Implement dynamic partition pruning #1072

Merged
merged 3 commits into from
Jun 10, 2020
Merged
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 @@ -199,7 +199,7 @@ public OperatorFactory duplicate()

private Split getLocalQuerySplit(Session session, TableHandle handle)
{
SplitSource splitSource = localQueryRunner.getSplitManager().getSplits(session, handle, UNGROUPED_SCHEDULING);
SplitSource splitSource = localQueryRunner.getSplitManager().getSplits(session, handle, UNGROUPED_SCHEDULING, TupleDomain::all);
List<Split> splits = new ArrayList<>();
while (!splitSource.isFinished()) {
splits.addAll(getNextBatch(splitSource));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import io.prestosql.spi.connector.ColumnHandle;
import io.prestosql.spi.connector.ConnectorSession;
import io.prestosql.spi.predicate.TupleDomain;
import io.prestosql.spi.type.TypeManager;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
Expand Down Expand Up @@ -71,7 +72,9 @@
import java.util.concurrent.Executor;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.BooleanSupplier;
import java.util.function.IntPredicate;
import java.util.function.Supplier;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

Expand All @@ -85,6 +88,7 @@
import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA;
import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE;
import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR;
import static io.prestosql.plugin.hive.HivePartitionManager.partitionMatches;
import static io.prestosql.plugin.hive.HiveSessionProperties.isForceLocalScheduling;
import static io.prestosql.plugin.hive.metastore.MetastoreUtil.getHiveSchema;
import static io.prestosql.plugin.hive.metastore.MetastoreUtil.getPartitionLocation;
Expand All @@ -97,6 +101,7 @@
import static io.prestosql.plugin.hive.util.HiveUtil.getFooterCount;
import static io.prestosql.plugin.hive.util.HiveUtil.getHeaderCount;
import static io.prestosql.plugin.hive.util.HiveUtil.getInputFormat;
import static io.prestosql.plugin.hive.util.HiveUtil.getPartitionKeyColumnHandles;
import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.lang.Integer.parseInt;
import static java.lang.Math.max;
Expand All @@ -119,6 +124,8 @@ public class BackgroundHiveSplitLoader

private final Table table;
private final TupleDomain<? extends ColumnHandle> compactEffectivePredicate;
private final Supplier<TupleDomain<ColumnHandle>> dynamicFilterSupplier;
private final TypeManager typeManager;
private final Optional<BucketSplitInfo> tableBucketInfo;
private final HdfsEnvironment hdfsEnvironment;
private final HdfsContext hdfsContext;
Expand Down Expand Up @@ -157,6 +164,8 @@ public BackgroundHiveSplitLoader(
Table table,
Iterable<HivePartitionMetadata> partitions,
TupleDomain<? extends ColumnHandle> compactEffectivePredicate,
Supplier<TupleDomain<ColumnHandle>> dynamicFilterSupplier,
TypeManager typeManager,
Optional<BucketSplitInfo> tableBucketInfo,
ConnectorSession session,
HdfsEnvironment hdfsEnvironment,
Expand All @@ -170,6 +179,8 @@ public BackgroundHiveSplitLoader(
{
this.table = table;
this.compactEffectivePredicate = compactEffectivePredicate;
this.dynamicFilterSupplier = dynamicFilterSupplier;
this.typeManager = typeManager;
this.tableBucketInfo = tableBucketInfo;
this.loaderConcurrency = loaderConcurrency;
this.session = session;
Expand Down Expand Up @@ -302,11 +313,19 @@ private ListenableFuture<?> loadSplits()
private ListenableFuture<?> loadPartition(HivePartitionMetadata partition)
throws IOException
{
String partitionName = partition.getHivePartition().getPartitionId();
HivePartition hivePartition = partition.getHivePartition();
String partitionName = hivePartition.getPartitionId();
Properties schema = getPartitionSchema(table, partition.getPartition());
List<HivePartitionKey> partitionKeys = getPartitionKeys(table, partition.getPartition());
TupleDomain<HiveColumnHandle> effectivePredicate = compactEffectivePredicate.transform(HiveColumnHandle.class::cast);

List<HiveColumnHandle> partitionColumns = getPartitionKeyColumnHandles(table, typeManager);
BooleanSupplier partitionMatchSupplier = () -> partitionMatches(partitionColumns, dynamicFilterSupplier.get(), hivePartition);
if (!partitionMatchSupplier.getAsBoolean()) {
// Avoid listing files and creating splits from a partition if it has been pruned due to dynamic filters
return COMPLETED_FUTURE;
}

Path path = new Path(getPartitionLocation(table, partition.getPartition()));
Configuration configuration = hdfsEnvironment.getConfiguration(hdfsContext, path);
InputFormat<?, ?> inputFormat = getInputFormat(configuration, schema, false);
Expand Down Expand Up @@ -349,6 +368,7 @@ private ListenableFuture<?> loadPartition(HivePartitionMetadata partition)
schema,
partitionKeys,
effectivePredicate,
partitionMatchSupplier,
partition.getTableToPartitionMapping(),
Optional.empty(),
isForceLocalScheduling(session),
Expand Down Expand Up @@ -386,6 +406,7 @@ private ListenableFuture<?> loadPartition(HivePartitionMetadata partition)
schema,
partitionKeys,
effectivePredicate,
partitionMatchSupplier,
raunaqmorarka marked this conversation as resolved.
Show resolved Hide resolved
partition.getTableToPartitionMapping(),
bucketConversionRequiresWorkerParticipation ? bucketConversion : Optional.empty(),
isForceLocalScheduling(session),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -253,6 +253,14 @@ private Optional<HivePartition> parseValuesAndFilterPartition(

private boolean partitionMatches(List<HiveColumnHandle> partitionColumns, TupleDomain<ColumnHandle> constraintSummary, Predicate<Map<ColumnHandle, NullableValue>> constraint, HivePartition partition)
{
return partitionMatches(partitionColumns, constraintSummary, partition) && constraint.test(partition.getKeys());
}

public static boolean partitionMatches(List<HiveColumnHandle> partitionColumns, TupleDomain<ColumnHandle> constraintSummary, HivePartition partition)
{
if (constraintSummary.isNone()) {
return false;
}
Map<ColumnHandle, Domain> domains = constraintSummary.getDomains().get();
for (HiveColumnHandle column : partitionColumns) {
NullableValue value = partition.getKeys().get(column);
Expand All @@ -261,8 +269,7 @@ private boolean partitionMatches(List<HiveColumnHandle> partitionColumns, TupleD
return false;
}
}

return constraint.test(partition.getKeys());
return true;
}

private List<String> getFilteredPartitionNames(SemiTransactionalHiveMetastore metastore, HiveIdentity identity, SchemaTableName tableName, List<HiveColumnHandle> partitionKeys, TupleDomain<ColumnHandle> effectivePredicate)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import io.prestosql.plugin.hive.util.HiveBucketing.HiveBucketFilter;
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.VersionEmbedder;
import io.prestosql.spi.connector.ColumnHandle;
import io.prestosql.spi.connector.ConnectorSession;
import io.prestosql.spi.connector.ConnectorSplitManager;
import io.prestosql.spi.connector.ConnectorSplitSource;
Expand All @@ -37,6 +38,8 @@
import io.prestosql.spi.connector.FixedSplitSource;
import io.prestosql.spi.connector.SchemaTableName;
import io.prestosql.spi.connector.TableNotFoundException;
import io.prestosql.spi.predicate.TupleDomain;
import io.prestosql.spi.type.TypeManager;
import org.weakref.jmx.Managed;
import org.weakref.jmx.Nested;

Expand All @@ -51,6 +54,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.function.Function;
import java.util.function.Supplier;

import static com.google.common.base.MoreObjects.firstNonNull;
import static com.google.common.base.Preconditions.checkArgument;
Expand Down Expand Up @@ -99,6 +103,7 @@ public class HiveSplitManager
private final int maxSplitsPerSecond;
private final boolean recursiveDfsWalkerEnabled;
private final CounterStat highMemorySplitSourceCounter;
private final TypeManager typeManager;

@Inject
public HiveSplitManager(
Expand All @@ -110,7 +115,8 @@ public HiveSplitManager(
DirectoryLister directoryLister,
ExecutorService executorService,
VersionEmbedder versionEmbedder,
CoercionPolicy coercionPolicy)
CoercionPolicy coercionPolicy,
TypeManager typeManager)
{
this(
metastoreProvider,
Expand All @@ -128,7 +134,8 @@ public HiveSplitManager(
hiveConfig.getMaxInitialSplits(),
hiveConfig.getSplitLoaderConcurrency(),
hiveConfig.getMaxSplitsPerSecond(),
hiveConfig.getRecursiveDirWalkerEnabled());
hiveConfig.getRecursiveDirWalkerEnabled(),
typeManager);
}

public HiveSplitManager(
Expand All @@ -147,7 +154,8 @@ public HiveSplitManager(
int maxInitialSplits,
int splitLoaderConcurrency,
@Nullable Integer maxSplitsPerSecond,
boolean recursiveDfsWalkerEnabled)
boolean recursiveDfsWalkerEnabled,
TypeManager typeManager)
{
this.metastoreProvider = requireNonNull(metastoreProvider, "metastore is null");
this.partitionManager = requireNonNull(partitionManager, "partitionManager is null");
Expand All @@ -166,6 +174,7 @@ public HiveSplitManager(
this.splitLoaderConcurrency = splitLoaderConcurrency;
this.maxSplitsPerSecond = firstNonNull(maxSplitsPerSecond, Integer.MAX_VALUE);
this.recursiveDfsWalkerEnabled = recursiveDfsWalkerEnabled;
this.typeManager = requireNonNull(typeManager, "typeManager is null");
}

@Override
Expand All @@ -174,6 +183,17 @@ public ConnectorSplitSource getSplits(
ConnectorSession session,
ConnectorTableHandle tableHandle,
SplitSchedulingStrategy splitSchedulingStrategy)
{
return getSplits(transaction, session, tableHandle, splitSchedulingStrategy, TupleDomain::all);
}

@Override
public ConnectorSplitSource getSplits(
ConnectorTransactionHandle transaction,
ConnectorSession session,
ConnectorTableHandle tableHandle,
SplitSchedulingStrategy splitSchedulingStrategy,
Supplier<TupleDomain<ColumnHandle>> dynamicFilter)
{
HiveTableHandle hiveTable = (HiveTableHandle) tableHandle;
SchemaTableName tableName = hiveTable.getSchemaTableName();
Expand Down Expand Up @@ -215,6 +235,8 @@ public ConnectorSplitSource getSplits(
table,
hivePartitions,
hiveTable.getCompactEffectivePredicate(),
dynamicFilter,
typeManager,
createBucketSplitInfo(bucketHandle, bucketFilter),
session,
hdfsEnvironment,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import java.util.Optional;
import java.util.OptionalInt;
import java.util.Properties;
import java.util.function.BooleanSupplier;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.ImmutableList.toImmutableList;
Expand All @@ -59,6 +60,7 @@ public class InternalHiveSplitFactory
private final List<HivePartitionKey> partitionKeys;
private final Optional<Domain> pathDomain;
private final TableToPartitionMapping tableToPartitionMapping;
private final BooleanSupplier partitionMatchSupplier;
private final Optional<BucketConversion> bucketConversion;
private final boolean forceLocalScheduling;
private final boolean s3SelectPushdownEnabled;
Expand All @@ -70,6 +72,7 @@ public InternalHiveSplitFactory(
Properties schema,
List<HivePartitionKey> partitionKeys,
TupleDomain<HiveColumnHandle> effectivePredicate,
BooleanSupplier partitionMatchSupplier,
TableToPartitionMapping tableToPartitionMapping,
Optional<BucketConversion> bucketConversion,
boolean forceLocalScheduling,
Expand All @@ -81,6 +84,7 @@ public InternalHiveSplitFactory(
this.schema = requireNonNull(schema, "schema is null");
this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null");
pathDomain = getPathDomain(requireNonNull(effectivePredicate, "effectivePredicate is null"));
this.partitionMatchSupplier = requireNonNull(partitionMatchSupplier, "partitionMatchSupplier is null");
this.tableToPartitionMapping = requireNonNull(tableToPartitionMapping, "tableToPartitionMapping is null");
this.bucketConversion = requireNonNull(bucketConversion, "bucketConversion is null");
this.forceLocalScheduling = forceLocalScheduling;
Expand Down Expand Up @@ -139,6 +143,12 @@ private Optional<InternalHiveSplit> createInternalHiveSplit(
return Optional.empty();
}

// Dynamic filter may not have been ready when partition was loaded in BackgroundHiveSplitLoader,
// but it might be ready when splits are enumerated lazily.
if (!partitionMatchSupplier.getAsBoolean()) {
raunaqmorarka marked this conversation as resolved.
Show resolved Hide resolved
return Optional.empty();
}

boolean forceLocalScheduling = this.forceLocalScheduling;

// For empty files, some filesystem (e.g. LocalFileSystem) produce one empty block
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -786,7 +786,8 @@ protected final void setup(String databaseName, HiveConfig hiveConfig, HiveMetas
hiveConfig.getMaxInitialSplits(),
hiveConfig.getSplitLoaderConcurrency(),
hiveConfig.getMaxSplitsPerSecond(),
false);
false,
TYPE_MANAGER);
pageSinkProvider = new HivePageSinkProvider(
getDefaultHiveFileWriterFactories(hiveConfig, hdfsEnvironment),
hdfsEnvironment,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,8 @@ protected void setup(String host, int port, String databaseName, boolean s3Selec
config.getMaxInitialSplits(),
config.getSplitLoaderConcurrency(),
config.getMaxSplitsPerSecond(),
config.getRecursiveDirWalkerEnabled());
config.getRecursiveDirWalkerEnabled(),
TYPE_MANAGER);
pageSinkProvider = new HivePageSinkProvider(
getDefaultHiveFileWriterFactories(config, hdfsEnvironment),
hdfsEnvironment,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -361,6 +361,8 @@ public HivePartitionMetadata next()
}
},
TupleDomain.all(),
TupleDomain::all,
TYPE_MANAGER,
createBucketSplitInfo(Optional.empty(), Optional.empty()),
SESSION,
new TestingHdfsEnvironment(TEST_FILES),
Expand Down Expand Up @@ -645,6 +647,8 @@ private static BackgroundHiveSplitLoader backgroundHiveSplitLoader(
table,
hivePartitionMetadatas,
compactEffectivePredicate,
TupleDomain::all,
TYPE_MANAGER,
createBucketSplitInfo(bucketHandle, hiveBucketFilter),
SESSION,
hdfsEnvironment,
Expand Down Expand Up @@ -672,6 +676,8 @@ private static BackgroundHiveSplitLoader backgroundHiveSplitLoader(List<LocatedF
SIMPLE_TABLE,
hivePartitionMetadatas,
TupleDomain.none(),
TupleDomain::all,
TYPE_MANAGER,
Optional.empty(),
connectorSession,
new TestingHdfsEnvironment(files),
Expand All @@ -693,6 +699,8 @@ private static BackgroundHiveSplitLoader backgroundHiveSplitLoaderOfflinePartiti
SIMPLE_TABLE,
createPartitionMetadataWithOfflinePartitions(),
TupleDomain.all(),
TupleDomain::all,
TYPE_MANAGER,
createBucketSplitInfo(Optional.empty(), Optional.empty()),
connectorSession,
new TestingHdfsEnvironment(TEST_FILES),
Expand Down
Loading