huaxingao commented on code in PR #5872:
URL: https://github.com/apache/iceberg/pull/5872#discussion_r988618124
##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java:
##########
@@ -145,6 +164,78 @@ public Filter[] pushedFilters() {
return pushedFilters;
}
+ @Override
+ public boolean pushAggregation(Aggregation aggregation) {
+ if (!(table instanceof BaseTable)) {
+ return false;
+ }
+ boolean aggregatePushdown =
+ Boolean.parseBoolean(
+ table
+ .properties()
+ .getOrDefault(AGGREGATE_PUSHDOWN_ENABLED,
AGGREGATE_PUSHDOWN_ENABLED_DEFAULT));
+ if (!aggregatePushdown) {
+ return false;
+ }
+
+ String deleteMode = table.properties().getOrDefault(DELETE_MODE,
DELETE_MODE_DEFAULT);
+ String updateMode = table.properties().getOrDefault(UPDATE_MODE,
UPDATE_MODE_DEFAULT);
+ String mergeMode = table.properties().getOrDefault(MERGE_MODE,
MERGE_MODE_DEFAULT);
+ // the statistics might be changed for merge on read and can't be used to
calculate
+ // min/max/count, so disable aggregate push down
+ // Todo: enable aggregate push down if there are not deletes files for
merge-on-read
+ if (deleteMode.equals("merge-on-read")
+ || updateMode.equals("merge-on-read")
+ || mergeMode.equals("merge-on-read")) {
+ return false;
+ }
Review Comment:
I changed this to
```
Map<String, String> map = table.currentSnapshot().summary();
if (Integer.parseInt(map.get("total-position-deletes")) > 0
|| Integer.parseInt(map.get("total-equality-deletes")) > 0) {
return false;
}
```
I am not sure if this is correct.
I added a test for this, but somehow the test failed at delete. I ignore the
test for now. Will figure out why the delete failed.
##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java:
##########
@@ -233,6 +324,16 @@ public Scan build() {
}
}
+ // if aggregates are pushed down, instead of constructing a
SparkBatchQueryScan, creating file
+ // read tasks and sending over the tasks to Spark executors, a
SparkLocalScan will be created
+ // and the scan is done locally on the Spark driver instead of the
executors. The statistics
+ // info will be retrieved from manifest file and used to build a Spark
internal row, which
+ // contains the pushed down aggregate values.
+ if (pushedAggregateRows != null) {
+ return new SparkLocalScan(
+ table, aggregateExpressions, pushedAggregateSchema,
pushedAggregateRows);
+ }
+
Review Comment:
Fixed. Thanks!
##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPushedDownAggregateUtil.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundAggregate;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.ArrayType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.MapType;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import scala.collection.JavaConverters;
+
+/** Helper methods for working with Spark aggregate push down. */
+public class SparkPushedDownAggregateUtil {
+
+ private SparkPushedDownAggregateUtil() {}
+
+ // Build schema for pushed down aggregates. This schema will be used as the
scan schema.
+ public static StructType buildSchemaForPushedDownAggregate(
+ List<Expression> aggregates, boolean caseSensitive, Schema schema) {
+ StructType finalSchema = new StructType();
+ for (int index = 0; index < aggregates.size(); index++) {
+ if ((aggregates.get(index)).op().name().equals("COUNTSTAR")) {
+ StructField field =
+ new StructField("COUNT(*)", DataTypes.LongType, false,
Metadata.empty());
+ finalSchema = finalSchema.add(field);
+ } else {
+ String colName = ((UnboundAggregate)
aggregates.get(index)).ref().name();
+ DataType dataType = getDataTypeForAggregateColumn(colName,
caseSensitive, schema);
+ if (dataType instanceof StructType
+ || dataType instanceof ArrayType
+ || dataType instanceof MapType) {
+ // not building pushed down aggregate schema for complex types to
disable aggregate push
+ // down because the statistic info for complex are not available.
+ return finalSchema;
+ }
+ if ((aggregates.get(index)).op().name().equals("COUNT")) {
+ StructField field =
+ new StructField(
+ "COUNT(" + colName + ")", DataTypes.LongType, false,
Metadata.empty());
+ finalSchema = finalSchema.add(field);
+ } else if ((aggregates.get(index)).op().name().equals("MAX")) {
+ StructField field =
+ new StructField("MAX(" + colName + ")", dataType, false,
Metadata.empty());
+ finalSchema = finalSchema.add(field);
+ } else if ((aggregates.get(index)).op().name().equals("MIN")) {
+ StructField field =
+ new StructField("MIN(" + colName + ")", dataType, false,
Metadata.empty());
+ finalSchema = finalSchema.add(field);
+ }
+ }
+ }
+ return finalSchema;
+ }
+
+ public static DataType getDataTypeForAggregateColumn(
+ String colName, boolean caseSensitive, Schema schema) {
+ Type type = null;
+ for (int i = 0; i < schema.columns().size(); i++) {
+ if ((caseSensitive && schema.columns().get(i).name().equals(colName))
+ || (!caseSensitive &&
schema.columns().get(i).name().equalsIgnoreCase(colName))) {
+ type = schema.columns().get(i).type();
+ }
+ }
+ return SparkSchemaUtil.convert(type);
+ }
+
+ @SuppressWarnings("checkstyle:CyclomaticComplexity")
+ public static InternalRow[] constructInternalRowForPushedDownAggregate(
Review Comment:
Added. Thanks
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]