Skip to content

showcase, DO NOT MERGE #14876

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -21,12 +21,13 @@ import java.nio.charset.StandardCharsets
import java.sql.Timestamp

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext}
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.aggregate.MergePartialAggregate
import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec}
import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange}
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -100,6 +101,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
python.ExtractPythonUDFs,
PlanSubqueries(sparkSession),
EnsureRequirements(sparkSession.sessionState.conf),
MergePartialAggregate,
CollapseCodegenStages(sparkSession.sessionState.conf),
ReuseExchange(sparkSession.sessionState.conf),
ReuseSubquery(sparkSession.sessionState.conf))
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.execution.aggregate

import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.SparkPlan

object MergePartialAggregate extends Rule[SparkPlan] {

override def apply(plan: SparkPlan): SparkPlan = plan transform {
// Normal partial aggregate pair
case outer @ HashAggregateExec(_, _, _, _, _, _, inner: HashAggregateExec)
if outer.aggregateExpressions.forall(_.mode == Final) &&
inner.aggregateExpressions.forall(_.mode == Partial) =>
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 we need to have more strict conditions to make sure these two operators are for the same group by clause (Although I do not have a case showing this will break, it is better to list the condition in a more specific way).

inner.copy(
aggregateExpressions = inner.aggregateExpressions.map(_.copy(mode = Complete)),
aggregateAttributes = inner.aggregateExpressions.map(_.resultAttribute),
resultExpressions = outer.resultExpressions)

// First partial aggregate pair for aggregation with distinct
case outer @ HashAggregateExec(_, _, _, _, _, _, inner: HashAggregateExec)
if outer.aggregateExpressions.forall(_.mode == PartialMerge) &&
inner.aggregateExpressions.forall(_.mode == Partial) =>
inner

// Second partial aggregate pair for aggregation with distinct.
// This is actually a no-op. For aggregation with distinct, the output of first partial
// aggregate is partitioned by grouping expressions and distinct attributes, and the second
// partial aggregate requires input to be partitioned by grouping attributes, which is not
// satisfied. `EnsureRequirements` will always insert exchange between these 2 aggregate exec
// and we will never hit this branch.
case outer @ HashAggregateExec(_, _, _, _, _, _, inner: HashAggregateExec)
if outer.aggregateExpressions.forall(_.mode == Final) &&
inner.aggregateExpressions.forall(_.mode == PartialMerge) =>
outer.copy(child = inner.child)

// Add similar logic for sort aggregate
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,21 +18,21 @@
package org.apache.spark.sql.execution

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{execution, Row}
import org.apache.spark.sql.{execution, QueryTest, Row}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Literal, SortOrder}
import org.apache.spark.sql.catalyst.plans.Inner
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Repartition}
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchange}
import org.apache.spark.sql.execution.exchange._
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._

class PlannerSuite extends SharedSQLContext {
class PlannerSuite extends QueryTest with SharedSQLContext {
import testImplicits._

setupTestData()
Expand Down Expand Up @@ -518,6 +518,24 @@ class PlannerSuite extends SharedSQLContext {
fail(s"Should have only two shuffles:\n$outputPlan")
}
}

test("no partial aggregation if input relation is already partitioned") {
val input = Seq("a" -> 1, "b" -> 2).toDF("i", "j")

val aggWithoutDistinct = input.repartition($"i").groupBy($"i").agg(sum($"j"))
checkAnswer(aggWithoutDistinct, input.groupBy($"i").agg(sum($"j")))
val numShuffles = aggWithoutDistinct.queryExecution.executedPlan.collect {
case e: Exchange => e
}.length
assert(numShuffles == 1)

val aggWithDistinct = input.repartition($"i", $"j").groupBy($"i").agg(countDistinct($"j"))
checkAnswer(aggWithDistinct, input.groupBy($"i").agg(countDistinct($"j")))
val numShuffles2 = aggWithDistinct.queryExecution.executedPlan.collect {
case e: Exchange => e
}.length
assert(numShuffles2 == 2)
}
}

// Used for unit-testing EnsureRequirements
Expand Down