-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Prune unused stats columns when reading Delta checkpoint #19848
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -68,11 +68,15 @@ | |
| import java.util.concurrent.TimeUnit; | ||
| import java.util.function.BiFunction; | ||
| import java.util.function.Function; | ||
| import java.util.function.Predicate; | ||
| import java.util.stream.Stream; | ||
|
|
||
| import static com.google.common.base.Predicates.alwaysFalse; | ||
| import static com.google.common.base.Predicates.alwaysTrue; | ||
| import static com.google.common.base.Throwables.throwIfUnchecked; | ||
| import static com.google.common.collect.ImmutableList.toImmutableList; | ||
| import static com.google.common.collect.ImmutableMap.toImmutableMap; | ||
| import static com.google.common.collect.ImmutableSet.toImmutableSet; | ||
| import static io.airlift.slice.SizeOf.estimatedSizeOf; | ||
| import static io.airlift.slice.SizeOf.instanceSize; | ||
| import static io.trino.cache.CacheUtils.invalidateAllIf; | ||
|
|
@@ -223,17 +227,48 @@ public MetadataEntry getMetadataEntry(TableSnapshot tableSnapshot, ConnectorSess | |
| .orElseThrow(() -> new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Metadata not found in transaction log for " + tableSnapshot.getTable())); | ||
| } | ||
|
|
||
| // Deprecated in favor of the namesake method which allows checkpoint filtering | ||
| // to be able to perform partition pruning and stats projection on the `add` entries | ||
| // from the checkpoint. | ||
| /** | ||
| * @see #getActiveFiles(TableSnapshot, MetadataEntry, ProtocolEntry, TupleDomain, Optional, ConnectorSession) | ||
| */ | ||
| @Deprecated | ||
| public List<AddFileEntry> getActiveFiles(TableSnapshot tableSnapshot, MetadataEntry metadataEntry, ProtocolEntry protocolEntry, ConnectorSession session) | ||
| { | ||
| return getActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), session); | ||
| return retrieveActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), Optional.empty(), session); | ||
| } | ||
|
|
||
| public List<AddFileEntry> getActiveFiles( | ||
| TableSnapshot tableSnapshot, | ||
| MetadataEntry metadataEntry, | ||
| ProtocolEntry protocolEntry, | ||
| TupleDomain<DeltaLakeColumnHandle> partitionConstraint, | ||
| Optional<Set<DeltaLakeColumnHandle>> projectedColumns, | ||
| ConnectorSession session) | ||
| { | ||
| Optional<Predicate<String>> addStatsMinMaxColumnFilter = Optional.of(alwaysFalse()); | ||
| if (projectedColumns.isPresent()) { | ||
| Set<String> baseColumnNames = projectedColumns.get().stream() | ||
| .filter(DeltaLakeColumnHandle::isBaseColumn) // Only base column stats are supported | ||
| .map(DeltaLakeColumnHandle::getColumnName) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Normalized (lowercased) or not?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We are using here the original column names - e.g See corresponding test |
||
| .collect(toImmutableSet()); | ||
| addStatsMinMaxColumnFilter = Optional.of(baseColumnNames::contains); | ||
| } | ||
| return retrieveActiveFiles(tableSnapshot, metadataEntry, protocolEntry, partitionConstraint, addStatsMinMaxColumnFilter, session); | ||
| } | ||
|
|
||
| public List<AddFileEntry> getActiveFiles(TableSnapshot tableSnapshot, MetadataEntry metadataEntry, ProtocolEntry protocolEntry, TupleDomain<DeltaLakeColumnHandle> partitionConstraint, ConnectorSession session) | ||
| private List<AddFileEntry> retrieveActiveFiles( | ||
| TableSnapshot tableSnapshot, | ||
| MetadataEntry metadataEntry, | ||
| ProtocolEntry protocolEntry, | ||
| TupleDomain<DeltaLakeColumnHandle> partitionConstraint, | ||
| Optional<Predicate<String>> addStatsMinMaxColumnFilter, | ||
| ConnectorSession session) | ||
| { | ||
| try { | ||
| if (isCheckpointFilteringEnabled(session)) { | ||
| return loadActiveFiles(tableSnapshot, metadataEntry, protocolEntry, partitionConstraint, session).stream() | ||
| return loadActiveFiles(tableSnapshot, metadataEntry, protocolEntry, partitionConstraint, addStatsMinMaxColumnFilter, session).stream() | ||
| .collect(toImmutableList()); | ||
| } | ||
|
|
||
|
|
@@ -264,7 +299,7 @@ public List<AddFileEntry> getActiveFiles(TableSnapshot tableSnapshot, MetadataEn | |
| } | ||
| } | ||
|
|
||
| List<AddFileEntry> activeFiles = loadActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), session); | ||
| List<AddFileEntry> activeFiles = loadActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), Optional.of(alwaysTrue()), session); | ||
| return new DeltaLakeDataFileCacheEntry(tableSnapshot.getVersion(), activeFiles); | ||
| }); | ||
| return cacheEntry.getActiveFiles(); | ||
|
|
@@ -279,6 +314,7 @@ private List<AddFileEntry> loadActiveFiles( | |
| MetadataEntry metadataEntry, | ||
| ProtocolEntry protocolEntry, | ||
| TupleDomain<DeltaLakeColumnHandle> partitionConstraint, | ||
| Optional<Predicate<String>> addStatsMinMaxColumnFilter, | ||
| ConnectorSession session) | ||
| { | ||
| List<Transaction> transactions = tableSnapshot.getTransactions(); | ||
|
|
@@ -290,7 +326,8 @@ private List<AddFileEntry> loadActiveFiles( | |
| fileSystemFactory.create(session), | ||
| fileFormatDataSourceStats, | ||
| Optional.of(new MetadataAndProtocolEntry(metadataEntry, protocolEntry)), | ||
| partitionConstraint)) { | ||
| partitionConstraint, | ||
| addStatsMinMaxColumnFilter)) { | ||
| return activeAddEntries(checkpointEntries, transactions) | ||
| .filter(partitionConstraint.isAll() | ||
| ? addAction -> true | ||
|
|
@@ -433,7 +470,7 @@ private <T> Stream<T> getEntries( | |
| List<Transaction> transactions = tableSnapshot.getTransactions(); | ||
| // Passing TupleDomain.all() because this method is used for getting all entries | ||
| Stream<DeltaLakeTransactionLogEntry> checkpointEntries = tableSnapshot.getCheckpointTransactionLogEntries( | ||
| session, entryTypes, checkpointSchemaManager, typeManager, fileSystem, stats, Optional.empty(), TupleDomain.all()); | ||
| session, entryTypes, checkpointSchemaManager, typeManager, fileSystem, stats, Optional.empty(), TupleDomain.all(), Optional.of(alwaysTrue())); | ||
|
|
||
| return entryMapper.apply( | ||
| checkpointEntries, | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.