diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index ad23c02fbcf1..193e4db258fb 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -484,8 +484,7 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con Supplier> lazyFiles = Suppliers.memoize(() -> { TableScan tableScan = icebergTable.newScan() .useSnapshot(table.getSnapshotId().get()) - .filter(toIcebergExpression(enforcedPredicate)) - .includeColumnStats(); + .filter(toIcebergExpression(enforcedPredicate)); try (CloseableIterable iterator = tableScan.planFiles()) { return ImmutableList.copyOf(iterator); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index 037c19d39ea8..028278d43152 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -163,12 +163,14 @@ public CompletableFuture 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(); this.pushedDownDynamicFilterPredicate = dynamicFilter.getCurrentPredicate().transformKeys(IcebergColumnHandle.class::cast); TupleDomain fullPredicate = tableHandle.getUnenforcedPredicate() .intersect(pushedDownDynamicFilterPredicate); // TODO: (https://github.com/trinodb/trino/issues/9743): Consider removing TupleDomain#simplify TupleDomain 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(); } @@ -182,11 +184,13 @@ public CompletableFuture 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);