huaxingao commented on code in PR #6622: URL: https://github.com/apache/iceberg/pull/6622#discussion_r1088489642
########## spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java: ########## @@ -158,6 +182,141 @@ public Filter[] pushedFilters() { return pushedFilters; } + @Override + public boolean pushAggregation(Aggregation aggregation) { + if (!pushDownAggregate(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 (Exception e) { + LOG.info("Can't push down aggregates: " + e.getMessage()); + return false; + } + + if (!metricsModeSupportsAggregatePushDown(aggregateEvaluator.aggregates())) { + LOG.info("The MetricsMode doesn't support aggregate push down."); + return false; + } + + List<ManifestFile> manifests = getSnapshot().allManifests(table.io()); + + for (ManifestFile manifest : manifests) { + try (ManifestReader<DataFile> reader = ManifestFiles.read(manifest, table.io())) { + for (DataFile dataFile : reader) { + aggregateEvaluator.update(dataFile.copy()); + } + } catch (IOException e) { + LOG.info("Can't push down aggregates: " + e.getMessage()); + return false; + } + } + + Object[] res = aggregateEvaluator.result(); + applyDataTypeConversionIfNecessary(res); + + List<Object> valuesInSparkInternalRow = java.util.Arrays.asList(res); + this.pushedAggregateRows = new InternalRow[1]; + pushedAggregateRows[0] = + InternalRow.fromSeq(JavaConverters.asScalaBuffer(valuesInSparkInternalRow).toSeq()); + pushedAggregateSchema = + SparkSchemaUtil.convert(new Schema(aggregateEvaluator.resultType().fields())); + return true; + } + + private boolean pushDownAggregate(Aggregation aggregation) { + if (!(table instanceof BaseTable)) { + return false; + } + + if (!readConf.aggregatePushDown()) { + return false; + } + + Snapshot snapshot = getSnapshot(); + if (snapshot == null) { + return false; + } else { + Map<String, String> map = snapshot.summary(); + // if there are row-level deletes in current snapshot, the statics + // maybe changed, so disable push down aggregate. + if (Integer.parseInt(map.getOrDefault("total-position-deletes", "0")) > 0 + || Integer.parseInt(map.getOrDefault("total-equality-deletes", "0")) > 0) { + LOG.info("Cannot push down aggregate (row-level deletes might change the statistics.)"); + return false; + } + } + + // If group by expression is the same as the partition, the statistics information can still + // be used to calculate min/max/count, will enable aggregate push down in next phase. + // TODO: enable aggregate push down for partition col group by expression + if (aggregation.groupByExpressions().length > 0) { + LOG.info("Cannot push down aggregate (group by is not supported yet)."); + return false; + } + + return true; + } + + private Snapshot getSnapshot() { + Snapshot snapshot = null; + if (readConf.snapshotId() != null) { + snapshot = table.snapshot(readConf.snapshotId()); + } else { + snapshot = table.currentSnapshot(); + } + + return snapshot; + } + + private void applyDataTypeConversionIfNecessary(Object[] result) { + for (int i = 0; i < result.length; i++) { + if (result[i] instanceof java.math.BigDecimal) { + result[i] = Decimal.apply(new scala.math.BigDecimal((BigDecimal) result[i])); + } else if (result[i] instanceof ByteBuffer) { + byte[] arr = new byte[((ByteBuffer) result[i]).remaining()]; + ((ByteBuffer) result[i]).get(arr); + result[i] = arr; + } else if (result[i] instanceof CharBuffer) { + result[i] = org.apache.spark.unsafe.types.UTF8String.fromString(result[i].toString()); + } + } + } + + private boolean metricsModeSupportsAggregatePushDown(List<BoundAggregate<?, ?>> aggregates) { + MetricsConfig config = MetricsConfig.forTable(table); + for (BoundAggregate aggregate : aggregates) { + String colName = aggregate.columnName(); + if (!colName.equals("*")) { + MetricsModes.MetricsMode mode = config.columnMode(colName); + if (mode.toString().equals("none")) { + return false; + } else if (mode.toString().equals("counts")) { + if (aggregate.op() == Expression.Operation.MAX + || aggregate.op() == Expression.Operation.MIN) { + return false; + } + } else if (mode.toString().contains("truncate")) { Review Comment: Agree. Addressed this and the above comments. -- 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