sunchao commented on a change in pull request #33352: URL: https://github.com/apache/spark/pull/33352#discussion_r676960998
########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Count.java ########## @@ -0,0 +1,49 @@ +/* + * 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.spark.sql.connector.expressions; + +import org.apache.spark.annotation.Evolving; + +/** + * An aggregate function that returns the number of the specific row in a group. + * + * @since 3.2.0 + */ +@Evolving +public final class Count implements AggregateFunc { + private FieldReference column; + private boolean isDistinct; + + public Count(FieldReference column, boolean isDistinct) { + this.column = column; Review comment: 2 space indentation? ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala ########## @@ -17,61 +17,179 @@ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, Expression, NamedExpression, PredicateHelper, ProjectionOverSchema, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.planning.ScanOperation -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.read.{Scan, V1Scan} +import org.apache.spark.sql.connector.expressions.Aggregation +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType -object V2ScanRelationPushDown extends Rule[LogicalPlan] { +object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper { import DataSourceV2Implicits._ - override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - case ScanOperation(project, filters, relation: DataSourceV2Relation) => - val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options) + def apply(plan: LogicalPlan): LogicalPlan = { + applyColumnPruning(pushdownAggregate(pushDownFilters(createScanBuilder(plan)))) + } + + private def createScanBuilder(plan: LogicalPlan) = plan.transform { + case r: DataSourceV2Relation => + ScanBuilderHolder(r.output, r, r.table.asReadable.newScanBuilder(r.options)) + } - val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output) + private def pushDownFilters(plan: LogicalPlan) = plan.transform { + // update the scan builder with filter push down and return a new plan with filter pushed + case Filter(condition, sHolder: ScanBuilderHolder) => + val filters = splitConjunctivePredicates(condition) + val normalizedFilters = + DataSourceStrategy.normalizeExprs(filters, sHolder.relation.output) val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) = normalizedFilters.partition(SubqueryExpression.hasSubquery) // `pushedFilters` will be pushed down and evaluated in the underlying data sources. // `postScanFilters` need to be evaluated after the scan. // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. val (pushedFilters, postScanFiltersWithoutSubquery) = PushDownUtils.pushFilters( - scanBuilder, normalizedFiltersWithoutSubquery) + sHolder.builder, normalizedFiltersWithoutSubquery) val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery + logInfo( + s""" + |Pushing operators to ${sHolder.relation.name} + |Pushed Filters: ${pushedFilters.mkString(", ")} + |Post-Scan Filters: ${postScanFilters.mkString(",")} + """.stripMargin) + + val filterCondition = postScanFilters.reduceLeftOption(And) + filterCondition.map(Filter(_, sHolder)).getOrElse(sHolder) + } + + def pushdownAggregate(plan: LogicalPlan): LogicalPlan = plan.transform { Review comment: nit: rename to `pushDownAggregate` to keep it consistent with `pushDownFilters`? ########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Count.java ########## @@ -0,0 +1,49 @@ +/* + * 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.spark.sql.connector.expressions; + +import org.apache.spark.annotation.Evolving; + +/** + * An aggregate function that returns the number of the specific row in a group. + * + * @since 3.2.0 + */ +@Evolving +public final class Count implements AggregateFunc { + private FieldReference column; Review comment: ditto: make these final ########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Count.java ########## @@ -0,0 +1,49 @@ +/* + * 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.spark.sql.connector.expressions; + +import org.apache.spark.annotation.Evolving; + +/** + * An aggregate function that returns the number of the specific row in a group. + * + * @since 3.2.0 + */ +@Evolving +public final class Count implements AggregateFunc { + private FieldReference column; + private boolean isDistinct; + + public Count(FieldReference column, boolean isDistinct) { + this.column = column; + this.isDistinct = isDistinct; + } + + public FieldReference column() { + return column; + } + public boolean isDinstinct() { Review comment: typo: `isDinstinct` -> `isDistinct`. ########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Aggregation.java ########## @@ -0,0 +1,46 @@ +/* + * 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.spark.sql.connector.expressions; + +import org.apache.spark.annotation.Evolving; + +import java.io.Serializable; + +/** + * Aggregation in SQL statement. + * + * @since 3.2.0 + */ +@Evolving +public final class Aggregation implements Serializable { + private AggregateFunc[] aggregateExpressions; Review comment: nit: mark these as `final`? ########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Sum.java ########## @@ -0,0 +1,57 @@ +/* + * 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.spark.sql.connector.expressions; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.types.DataType; + +/** + * An aggregate function that returns the summation of all the values in a group. + * + * @since 3.2.0 + */ +@Evolving +public final class Sum implements AggregateFunc { + private FieldReference column; + private DataType dataType; + private boolean isDistinct; + + public Sum(FieldReference column, DataType dataType, boolean isDistinct) { + this.column = column; + this.dataType = dataType; + this.isDistinct = isDistinct; + } + + public FieldReference column() { + return column; + } + public DataType dataType() { + return dataType; + } + public boolean isDinstinct() { Review comment: ditto ########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownAggregates.java ########## @@ -0,0 +1,56 @@ +/* + * 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.spark.sql.connector.read; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.expressions.Aggregation; + +/** + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to + * push down aggregates. Spark assumes that the data source can't fully complete the + * grouping work, and will group the data source output again. For queries like + * "SELECT min(value) AS m FROM t GROUP BY key", after pushing down the aggregate + * to the data source, the data source can still output data with duplicated keys, which is OK + * as Spark will do GROUP BY key again. The final query plan can be something like this: + * {{{ + * Aggregate [key#1], [min(min(value)#2) AS m#3] + * +- RelationV2[key#1, min(value)#2] + * }}} + * + * <p> + * Similarly, if there is no grouping expression, the data source can still output more than one + * rows. + * + * <p> + * When pushing down operators, Spark pushes down filters to the data source first, then push down + * aggregates or apply column pruning. Depends on data source implementation, aggregates may or + * may not be able to be pushed down with filters. If pushed filters still need to be evaluated + * after scanning, aggregates can't be pushed down. + * + * @since 3.2.0 + */ +@Evolving +public interface SupportsPushDownAggregates extends ScanBuilder { + + /** + * Pushes down Aggregation to datasource. The order of the datasource scan output columns should + * be: grouping columns, aggregate columns (in the same order as the aggregate functions in + * the given Aggregation). + */ + boolean pushAggregation(Aggregation aggregation); Review comment: +1. What is the returned boolean for? ########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownAggregates.java ########## @@ -0,0 +1,56 @@ +/* + * 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.spark.sql.connector.read; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.expressions.Aggregation; + +/** + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to + * push down aggregates. Spark assumes that the data source can't fully complete the + * grouping work, and will group the data source output again. For queries like + * "SELECT min(value) AS m FROM t GROUP BY key", after pushing down the aggregate + * to the data source, the data source can still output data with duplicated keys, which is OK + * as Spark will do GROUP BY key again. The final query plan can be something like this: + * {{{ Review comment: this is not properly rendered, you can use: ``` * <pre> * Aggregate [key#1], [min(min(value)#2) AS m#3] * +- RelationV2[key#1, min(value)#2] * </pre> * Similarly, if there is no grouping expression, the data source can still output more than one * rows. ``` instead. Note that the following `<p>` is also removed. ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala ########## @@ -70,6 +74,42 @@ object PushDownUtils extends PredicateHelper { } } + /** + * Pushes down aggregates to the data source reader + * + * @return pushed aggregation. + */ + def pushAggregates( + scanBuilder: ScanBuilder, + aggregates: Seq[AggregateExpression], + groupBy: Seq[Expression]): Option[Aggregation] = { + + def columnAsString(e: Expression): Option[FieldReference] = e match { + case PushableColumnWithoutNestedColumn(name) => + Some(FieldReference(name).asInstanceOf[FieldReference]) + case _ => None + } + + scanBuilder match { + case r: SupportsPushDownAggregates => + val translatedAggregates = aggregates.map(DataSourceStrategy.translateAggregate).flatten + val translatedGroupBys = groupBy.map(columnAsString).flatten + + if (translatedAggregates.length != aggregates.length || + translatedGroupBys.length != groupBy.length) { + return None + } + + val agg = new Aggregation(translatedAggregates.toArray, translatedGroupBys.toArray) + if (r.pushAggregation(agg)) { Review comment: nit: you can just use `Some(agg).filter(r.pushAggregation)` ########## File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Count.java ########## @@ -0,0 +1,49 @@ +/* + * 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.spark.sql.connector.expressions; + +import org.apache.spark.annotation.Evolving; + +/** + * An aggregate function that returns the number of the specific row in a group. + * + * @since 3.2.0 + */ +@Evolving +public final class Count implements AggregateFunc { + private FieldReference column; + private boolean isDistinct; + + public Count(FieldReference column, boolean isDistinct) { + this.column = column; + this.isDistinct = isDistinct; + } + + public FieldReference column() { + return column; Review comment: ditto ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala ########## @@ -70,6 +74,42 @@ object PushDownUtils extends PredicateHelper { } } + /** + * Pushes down aggregates to the data source reader + * + * @return pushed aggregation. + */ + def pushAggregates( + scanBuilder: ScanBuilder, + aggregates: Seq[AggregateExpression], + groupBy: Seq[Expression]): Option[Aggregation] = { + + def columnAsString(e: Expression): Option[FieldReference] = e match { + case PushableColumnWithoutNestedColumn(name) => + Some(FieldReference(name).asInstanceOf[FieldReference]) + case _ => None + } + + scanBuilder match { + case r: SupportsPushDownAggregates => + val translatedAggregates = aggregates.map(DataSourceStrategy.translateAggregate).flatten Review comment: nit: can use `flatMap` instead of `map` + `flatten`. -- 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]
