Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,8 @@
/**
* An interface for building the {@link Scan}. Implementations can mixin SupportsPushDownXYZ
* interfaces to do operator pushdown, and keep the operator pushdown result in the returned
* {@link Scan}.
* {@link Scan}. When pushing down operators, Spark pushes down filters first, then pushes down
* aggregates or applies column pruning.
*
* @since 3.0.0
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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 source can implement this interface to
* push down aggregates. Depends on the data source implementation, the aggregates may not
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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]
}}}

Similarly, if there is no grouping expression, the data source can still output more than one rows.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also let's use valid Java doc syntax, e.g., add <p> between paragraphs, properly format code blocks, etc.

* be able to push down, or partially push down and have a final aggregate at Spark.
* For example, "SELECT min(_1) FROM t GROUP BY _2" can be pushed down to data source,
* the partially aggregated result min(_1) grouped by _2 will be returned to Spark, and
* then have a final aggregation.
* {{{
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

* Aggregate [_2#10], [min(_2#10) AS min(_1)#16]
* +- RelationV2[_2#10, min(_1)#18]
* }}}
*
* 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);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For public API, we should document what the returned value means.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1. What is the returned boolean for?

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.sql.types.DataType

// Aggregate Functions in SQL statement.
// e.g. SELECT COUNT(EmployeeID), Max(salary) FROM dept GROUP BY deptID
// aggregateExpressions are (COUNT(EmployeeID), Max(salary)), groupByColumns are (deptID)
case class Aggregation(aggregateExpressions: Seq[AggregateFunc],
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is public DS v2 API, can we write it in Java?

groupByColumns: Seq[Expression])

abstract class AggregateFunc

case class Min(column: Expression, dataType: DataType) extends AggregateFunc
case class Max(column: Expression, dataType: DataType) extends AggregateFunc
case class Sum(column: Expression, dataType: DataType, isDistinct: Boolean)
extends AggregateFunc
case class Count(column: Expression, dataType: DataType, isDistinct: Boolean)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry to change my mind at the last second. I think it's very unlikely that a data source can support something like max(a + b), group by a + b. I think it's clearer to use NamedReference instead of Expression here.

For v2 partitioning, it's always named. e.g. CREATE TABLE ... PARTITIONED BY year(ts), the partitioning has a name and you can get it by DESC TABLE, which calls SupportsPartitionManagement.partitionSchema.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For count(1), let's create a special one class CountOne.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW count doesn't need a data type? It always returns long.

extends AggregateFunc
Original file line number Diff line number Diff line change
Expand Up @@ -870,6 +870,12 @@ object SQLConf {
.checkValue(threshold => threshold >= 0, "The threshold must not be negative.")
.createWithDefault(10)

val PARQUET_AGGREGATE_PUSHDOWN_ENABLED = buildConf("spark.sql.parquet.aggregatePushdown")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the config is per source. For this API-only PR, we don't need any config.

.doc("Enables Parquet aggregate push-down optimization when set to true.")
.version("3.2.0")
.booleanConf
.createWithDefault(false)

val PARQUET_WRITE_LEGACY_FORMAT = buildConf("spark.sql.parquet.writeLegacyFormat")
.doc("If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal " +
"values will be written in Apache Parquet's fixed-length byte array format, which other " +
Expand Down Expand Up @@ -3619,6 +3625,8 @@ class SQLConf extends Serializable with Logging {
def parquetFilterPushDownInFilterThreshold: Int =
getConf(PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD)

def parquetAggregatePushDown: Boolean = getConf(PARQUET_AGGREGATE_PUSHDOWN_ENABLED)

def orcFilterPushDown: Boolean = getConf(ORC_FILTER_PUSHDOWN_ENABLED)

def isOrcSchemaMergingEnabled: Boolean = getConf(ORC_SCHEMA_MERGING_ENABLED)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,14 @@ import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions._
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.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2
import org.apache.spark.sql.connector.catalog.SupportsRead
import org.apache.spark.sql.connector.catalog.TableCapability._
import org.apache.spark.sql.connector.expressions.{AggregateFunc, Count, FieldReference, LiteralValue, Max, Min}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.{InSubqueryExec, RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.command._
Expand Down Expand Up @@ -692,6 +694,29 @@ object DataSourceStrategy
(nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters)
}

protected[sql] def translateAggregate(aggregates: AggregateExpression): Option[AggregateFunc] = {
if (aggregates.filter.isEmpty) {
aggregates.aggregateFunction match {
case min@aggregate.Min(PushableColumnAndNestedColumn(name)) =>
Some(Min(FieldReference(Seq(name)), min.dataType))
case max@aggregate.Max(PushableColumnAndNestedColumn(name)) =>
Some(Max(FieldReference(Seq(name)), max.dataType))
case count: aggregate.Count if count.children.length == 1 =>
count.children.head match {
// SELECT COUNT(*) FROM table is translated to SELECT 1 FROM table
case Literal(_, _) =>
Some(Count(LiteralValue(1L, LongType), LongType, aggregates.isDistinct))
case PushableColumnAndNestedColumn(name) =>
Some(Count(FieldReference(Seq(name)), LongType, aggregates.isDistinct))
case _ => None
}
case _ => None
}
} else {
None
}
}

/**
* Convert RDD of Row into RDD of InternalRow with objects in catalyst types
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,10 @@ package org.apache.spark.sql.execution.datasources.v2
import scala.collection.mutable

import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, NamedExpression, PredicateHelper, SchemaPruning}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.connector.expressions.{Aggregation, FieldReference}
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -70,6 +73,41 @@ 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 AttributeReference(name, _, _, _) => Some(FieldReference(Seq(name)))
case _ => None
}

scanBuilder match {
case r: SupportsPushDownAggregates =>
val translatedAggregates = aggregates.map(DataSourceStrategy.translateAggregate).flatten
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: can use flatMap instead of map + flatten.

val translatedGroupBys = groupBy.map(columnAsString).flatten

if (translatedAggregates.length != aggregates.length ||
translatedGroupBys.length != groupBy.length) {
return None
}

val agg = Aggregation(translatedAggregates, translatedGroupBys)
if (r.pushAggregation(agg)) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: you can just use Some(agg).filter(r.pushAggregation)

Some(agg)
} else {
None
}
case _ => None
}
}

/**
* Applies column pruning to the data source, w.r.t. the references of the given expressions.
*
Expand Down
Loading