rdblue commented on code in PR #6622:
URL: https://github.com/apache/iceberg/pull/6622#discussion_r1112401419


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java:
##########
@@ -158,6 +178,134 @@ public Filter[] pushedFilters() {
     return pushedFilters;
   }
 
+  @Override
+  public boolean pushAggregation(Aggregation aggregation) {
+    if (!canPushDownAggregation(aggregation)) {
+      return false;
+    }
+
+    AggregateEvaluator aggregateEvaluator;
+    try {
+      List<Expression> aggregates =
+          Arrays.stream(aggregation.aggregateExpressions())
+              .map(agg -> SparkAggregates.convert(agg))
+              .collect(Collectors.toList());
+      aggregateEvaluator = AggregateEvaluator.create(schema, aggregates);
+    } catch (UnsupportedOperationException | IllegalArgumentException e) {
+      LOG.info("Skipped aggregate pushdown: " + e);
+      return false;
+    }
+
+    if 
(!metricsModeSupportsAggregatePushDown(aggregateEvaluator.aggregates())) {
+      return false;
+    }
+
+    TableScan scan = table.newScan();
+    ((DataTableScan) scan).setStats(true);
+    Snapshot snapshot = readSnapshot();
+    if (snapshot == null) {
+      LOG.info("Skipped aggregate pushdown: table snapshot is null");
+      return false;
+    }
+    scan = scan.useSnapshot(snapshot.snapshotId());
+    scan = configureSplitPlanning(scan);
+
+    try (CloseableIterable<FileScanTask> fileScanTasks = scan.planFiles()) {
+      List<FileScanTask> tasks = ImmutableList.copyOf(fileScanTasks);
+      for (FileScanTask task : tasks) {
+        if (!task.deletes().isEmpty()) {
+          LOG.info("Skipped aggregate pushdown: detected row level deletes");
+          return false;
+        }
+
+        aggregateEvaluator.update(task.file());
+      }
+    } catch (IOException e) {
+      LOG.info("Skipped aggregate pushdown: " + e);

Review Comment:
   This should not swallow the exception. It should log the entire exception 
before returning false.



-- 
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.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

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


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

Reply via email to