You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/04/02 22:49:32 UTC

[GitHub] [iceberg] szehon-ho commented on a change in pull request #2358: Core: Add predicate push down for Partitions table

szehon-ho commented on a change in pull request #2358:
URL: https://github.com/apache/iceberg/pull/2358#discussion_r606473839



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -98,21 +105,46 @@ private DataTask task(TableScan scan) {
     return StaticDataTask.Row.of(partition.key, partition.recordCount, partition.fileCount);
   }
 
-  private static Iterable<Partition> partitions(Table table, Long snapshotId) {
+  private Iterable<Partition> partitions(StaticTableScan scan) {
+    CloseableIterable<FileScanTask> tasks = planTasks(scan);
+
     PartitionMap partitions = new PartitionMap(table.spec().partitionType());
-    TableScan scan = table.newScan();
+    for (FileScanTask task : tasks) {
+      partitions.get(task.file().partition()).update(task.file());
+    }
+    return partitions.all();
+  }
 
-    if (snapshotId != null) {
-      scan = scan.useSnapshot(snapshotId);
+  @VisibleForTesting
+  CloseableIterable<FileScanTask> planTasks(StaticTableScan scan) {
+    boolean caseSensitive = scan.isCaseSensitive();
+    boolean ignoreResiduals = scan.shouldIgnoreResiduals();
+    long snapshotId = scan.snapshot().snapshotId();
+
+    ManifestEvaluator eval = ManifestEvaluator.forPartitionFilter(
+        Projections.metadata(table.spec(), caseSensitive).project(scan.filter()),

Review comment:
       Hi yes, it's basically this but it's a little more.  I should have explained a bit more my thinking in the PR.
   
   Beyond transforming predicate "partition.x" => "x", there's also logic to handle predicates that don't have to do with the partitions, for example "record_count" > 0.  It seems the other Projections (InclusiveProjection/StrictProjection) were handling this logic (returning false for non-partition predicates, and all the and/or/not, etc),so I thought to re-use it.  It seems we have to 'project' those out, as  ManifestEvaluator Constructor tries bind predicates to the table's partitionSpec, and non-partition predicates will fail to be bound.
   
   I can probably use a cloned partition spec as you said, but the problem would still remain, and I would have to write my own ExpressionVisitor that does the same logic?  Or let me know if I misunderstand.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org