Skip to content
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 @@ -484,8 +484,7 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con
Supplier<List<FileScanTask>> lazyFiles = Suppliers.memoize(() -> {
TableScan tableScan = icebergTable.newScan()
.useSnapshot(table.getSnapshotId().get())
.filter(toIcebergExpression(enforcedPredicate))
.includeColumnStats();
.filter(toIcebergExpression(enforcedPredicate));

try (CloseableIterable<FileScanTask> iterator = tableScan.planFiles()) {
return ImmutableList.copyOf(iterator);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,12 +163,14 @@ public CompletableFuture<ConnectorSplitBatch> getNextBatch(int maxSize)

if (fileScanTaskIterable == null) {
// Used to avoid duplicating work if the Dynamic Filter was already pushed down to the Iceberg API
boolean dynamicFilterIsComplete = dynamicFilter.isComplete();
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@raunaqmorarka can you sanity check for me that isComplete() means that the filter we get from getCurrentPredicate() will not change later in the query?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, that is right

this.pushedDownDynamicFilterPredicate = dynamicFilter.getCurrentPredicate().transformKeys(IcebergColumnHandle.class::cast);
TupleDomain<IcebergColumnHandle> fullPredicate = tableHandle.getUnenforcedPredicate()
.intersect(pushedDownDynamicFilterPredicate);
// TODO: (https://github.com/trinodb/trino/issues/9743): Consider removing TupleDomain#simplify
TupleDomain<IcebergColumnHandle> simplifiedPredicate = fullPredicate.simplify(ICEBERG_DOMAIN_COMPACTION_THRESHOLD);
if (!simplifiedPredicate.equals(fullPredicate)) {
boolean usedSimplifiedPredicate = !simplifiedPredicate.equals(fullPredicate);
if (usedSimplifiedPredicate) {
// Pushed down predicate was simplified, always evaluate it against individual splits
this.pushedDownDynamicFilterPredicate = TupleDomain.all();
}
Expand All @@ -182,11 +184,13 @@ public CompletableFuture<ConnectorSplitBatch> getNextBatch(int maxSize)
}

Expression filterExpression = toIcebergExpression(effectivePredicate);
this.fileScanTaskIterable = TableScanUtil.splitFiles(
tableScan.filter(filterExpression)
.includeColumnStats()
.planFiles(),
tableScan.targetSplitSize());
// If the Dynamic Filter will be evaluated against each file, stats are required. Otherwise, skip them.
boolean requiresColumnStats = usedSimplifiedPredicate || !dynamicFilterIsComplete;
TableScan scan = tableScan.filter(filterExpression);
if (requiresColumnStats) {
scan = scan.includeColumnStats();
}
this.fileScanTaskIterable = TableScanUtil.splitFiles(scan.planFiles(), tableScan.targetSplitSize());
closer.register(fileScanTaskIterable);
this.fileScanTaskIterator = fileScanTaskIterable.iterator();
closer.register(fileScanTaskIterator);
Expand Down