[SPARK-7440][SQL] Remove physical Distinct operator in favor of Aggregate

This patch replaces Distinct with Aggregate in the optimizer, so Distinct will become
more efficient over time as we optimize Aggregate (via Tungsten).

Author: Reynold Xin <rxin@databricks.com>

Closes #6637 from rxin/replace-distinct and squashes the following commits:

b3cc50e [Reynold Xin] Mima excludes.
93d6117 [Reynold Xin] Code review feedback.
87e4741 [Reynold Xin] [SPARK-7440][SQL] Remove physical Distinct operator in favor of Aggregate.
This commit is contained in:
Reynold Xin 2015-06-04 13:52:53 -07:00
parent 6593842271
commit 2bcdf8c239
7 changed files with 65 additions and 35 deletions

View file

@ -46,7 +46,9 @@ object MimaExcludes {
"org.apache.spark.api.java.JavaRDDLike.partitioner"),
// Mima false positive (was a private[spark] class)
ProblemFilters.exclude[MissingClassProblem](
"org.apache.spark.util.collection.PairIterator")
"org.apache.spark.util.collection.PairIterator"),
// SQL execution is considered private.
excludePackage("org.apache.spark.sql.execution")
)
case v if v.startsWith("1.4") =>
Seq(

View file

@ -36,6 +36,8 @@ object DefaultOptimizer extends Optimizer {
// SubQueries are only needed for analysis and can be removed before execution.
Batch("Remove SubQueries", FixedPoint(100),
EliminateSubQueries) ::
Batch("Distinct", FixedPoint(100),
ReplaceDistinctWithAggregate) ::
Batch("Operator Reordering", FixedPoint(100),
UnionPushdown,
CombineFilters,
@ -696,3 +698,15 @@ object ConvertToLocalRelation extends Rule[LogicalPlan] {
LocalRelation(projectList.map(_.toAttribute), data.map(projection))
}
}
/**
* Replaces logical [[Distinct]] operator with an [[Aggregate]] operator.
* {{{
* SELECT DISTINCT f1, f2 FROM t ==> SELECT f1, f2 FROM t GROUP BY f1, f2
* }}}
*/
object ReplaceDistinctWithAggregate extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case Distinct(child) => Aggregate(child.output, child.output, child)
}
}

View file

@ -339,6 +339,9 @@ case class Sample(
override def output: Seq[Attribute] = child.output
}
/**
* Returns a new logical plan that dedups input rows.
*/
case class Distinct(child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
}

View file

@ -0,0 +1,42 @@
/*
* 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.catalyst.optimizer
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Distinct, LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
class ReplaceDistinctWithAggregateSuite extends PlanTest {
object Optimize extends RuleExecutor[LogicalPlan] {
val batches = Batch("ProjectCollapsing", Once, ReplaceDistinctWithAggregate) :: Nil
}
test("replace distinct with aggregate") {
val input = LocalRelation('a.int, 'b.int)
val query = Distinct(input)
val optimized = Optimize.execute(query.analyze)
val correctAnswer = Aggregate(input.output, input.output, input)
comparePlans(optimized, correctAnswer)
}
}

View file

@ -1311,7 +1311,7 @@ class DataFrame private[sql](
* @group dfops
* @since 1.3.0
*/
override def distinct: DataFrame = Distinct(logicalPlan)
override def distinct: DataFrame = dropDuplicates()
/**
* @group basic

View file

@ -284,8 +284,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case r: RunnableCommand => ExecutedCommand(r) :: Nil
case logical.Distinct(child) =>
execution.Distinct(partial = false,
execution.Distinct(partial = true, planLater(child))) :: Nil
throw new IllegalStateException(
"logical distinct operator should have been replaced by aggregate in the optimizer")
case logical.Repartition(numPartitions, shuffle, child) =>
execution.Repartition(numPartitions, shuffle, planLater(child)) :: Nil
case logical.SortPartitions(sortExprs, child) =>

View file

@ -230,37 +230,6 @@ case class ExternalSort(
override def outputOrdering: Seq[SortOrder] = sortOrder
}
/**
* :: DeveloperApi ::
* Computes the set of distinct input rows using a HashSet.
* @param partial when true the distinct operation is performed partially, per partition, without
* shuffling the data.
* @param child the input query plan.
*/
@DeveloperApi
case class Distinct(partial: Boolean, child: SparkPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
override def requiredChildDistribution: Seq[Distribution] =
if (partial) UnspecifiedDistribution :: Nil else ClusteredDistribution(child.output) :: Nil
protected override def doExecute(): RDD[Row] = {
child.execute().mapPartitions { iter =>
val hashSet = new scala.collection.mutable.HashSet[Row]()
var currentRow: Row = null
while (iter.hasNext) {
currentRow = iter.next()
if (!hashSet.contains(currentRow)) {
hashSet.add(currentRow.copy())
}
}
hashSet.iterator
}
}
}
/**
* :: DeveloperApi ::
* Return a new RDD that has exactly `numPartitions` partitions.