Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
8d0256d
init work for parquet aggregate push down
Mar 26, 2021
3b7a170
change count
Mar 29, 2021
a4b054a
change countpwd
Mar 30, 2021
d576cf6
[SPARK-34952][SQL] Aggregate (Min/Max/Count) push down for Parquet
Apr 4, 2021
a66a87c
remove blank
huaxingao Apr 4, 2021
e561cc2
fix build failure
huaxingao Apr 4, 2021
d52a72c
address comments
huaxingao Apr 5, 2021
60b495d
minor
huaxingao Apr 5, 2021
d954727
throw Exception if statics is not available in parquet + code clean up
huaxingao Apr 12, 2021
a505e7f
fix build failure
huaxingao Apr 12, 2021
d118eaf
rewrite count to sum for pushed down count
huaxingao Apr 26, 2021
9af611f
remove unnessary change in Count
huaxingao Apr 26, 2021
d277701
add a rule for PartialAggregatePushDown
huaxingao Apr 28, 2021
743bc8a
fix tests failure
huaxingao Apr 28, 2021
346485e
add interface SupportsPushDownAggregates
huaxingao May 15, 2021
ed7c19d
fix java lint error
huaxingao May 15, 2021
52b0b98
add default case
huaxingao May 15, 2021
6df2ae1
address comments
huaxingao May 28, 2021
d0a61f8
add default case
huaxingao May 28, 2021
91f013c
address comments
huaxingao Jun 9, 2021
4fbe666
fix lint error
huaxingao Jun 9, 2021
a5833ef
not push down agg if it has timestamp (INT96 is sort order is undefined)
huaxingao Jun 16, 2021
a60c9a1
address comments
huaxingao Jun 18, 2021
475c0f7
change SupportsPushDownAggregates interface
huaxingao Jun 23, 2021
564e6de
fix lint-scala
huaxingao Jun 23, 2021
1648070
remove completely push down status
huaxingao Jul 2, 2021
ccdc543
address comments
huaxingao Jul 12, 2021
7540b59
address comments
huaxingao Jul 13, 2021
2c889c6
fix build failure:
huaxingao Jul 13, 2021
5c2b630
address comments
huaxingao Jul 14, 2021
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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 push down
* aggregates or apply 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
* 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.
* {{{
* 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);
}
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), deptID FROM dept GROUP BY deptID
// aggregateExpressions are (COUNT(EmployeeID), Max(salary)), groupByColumns are (deptID)
case class Aggregation(aggregateExpressions: Seq[AggregateFunc],
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)
extends AggregateFunc
Original file line number Diff line number Diff line change
Expand Up @@ -772,6 +772,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
Member

Choose a reason for hiding this comment

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

Just curious - would anyone ever not want to push it down?
I'm surprised, I thought we already did this!
CC @cloud-fan

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@srowen Hello Sean :)
Actually we only have filter push down for parquet, not aggregate push down yet. I will probably change the default to true after this PR gets reviewed and fully tested.

.doc("Enables Parquet aggregate push-down optimization when set to true.")
Copy link
Contributor

Choose a reason for hiding this comment

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

Shall it be an internal config or not? Do we expect this one to be user-facing and tune it frequently?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for reviewing!
I think this should be similar to PARQUET_FILTER_PUSHDOWN_ENABLED and be a user-facing config. I guess we can default it to true in the future after we have more testing.

.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 @@ -3423,6 +3429,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 @@ -37,7 +37,8 @@ class SparkOptimizer(

override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
// TODO: move SchemaPruning into catalyst
SchemaPruning :: V2ScanRelationPushDown :: V2Writes :: PruneFileSourcePartitions :: Nil
SchemaPruning :: V2ScanRelationPushDown :: V2Writes ::
PruneFileSourcePartitions :: Nil
Copy link
Member

Choose a reason for hiding this comment

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

nit: unnecessary change?


override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+
Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+
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.{RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.command._
Expand Down Expand Up @@ -673,6 +675,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
Loading