Skip to content
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 @@ -51,7 +51,7 @@ case class Exchange(
}

val simpleNodeName = if (tungstenMode) "TungstenExchange" else "Exchange"
s"${simpleNodeName}${extraInfo}"
s"$simpleNodeName$extraInfo"
}

/**
Expand Down Expand Up @@ -475,10 +475,7 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[
if (requiredOrdering.nonEmpty) {
// If child.outputOrdering is [a, b] and requiredOrdering is [a], we do not need to sort.
if (requiredOrdering != child.outputOrdering.take(requiredOrdering.length)) {
sqlContext.planner.BasicOperators.getSortOperator(
requiredOrdering,
global = false,
child)
Sort(requiredOrdering, global = false, child = child)
} else {
child
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,68 +17,22 @@

package org.apache.spark.sql.execution

import org.apache.spark.{InternalAccumulator, SparkEnv, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution}
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.CompletionIterator
import org.apache.spark.util.collection.ExternalSorter
import org.apache.spark.{InternalAccumulator, SparkEnv, TaskContext}

////////////////////////////////////////////////////////////////////////////////////////////////////
// This file defines various sort operators.
////////////////////////////////////////////////////////////////////////////////////////////////////

/**
* Performs a sort, spilling to disk as needed.
* @param global when true performs a global sort of all partitions by shuffling the data first
* if necessary.
*/
case class Sort(
sortOrder: Seq[SortOrder],
global: Boolean,
child: SparkPlan)
extends UnaryNode {

override def requiredChildDistribution: Seq[Distribution] =
if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil

protected override def doExecute(): RDD[InternalRow] = attachTree(this, "sort") {
child.execute().mapPartitionsInternal( { iterator =>
val ordering = newOrdering(sortOrder, child.output)
val sorter = new ExternalSorter[InternalRow, Null, InternalRow](
TaskContext.get(), ordering = Some(ordering))
sorter.insertAll(iterator.map(r => (r.copy(), null)))
val baseIterator = sorter.iterator.map(_._1)
val context = TaskContext.get()
context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled)
context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled)
context.internalMetricsToAccumulators(
InternalAccumulator.PEAK_EXECUTION_MEMORY).add(sorter.peakMemoryUsedBytes)
// TODO(marmbrus): The complex type signature below thwarts inference for no reason.
CompletionIterator[InternalRow, Iterator[InternalRow]](baseIterator, sorter.stop())
}, preservesPartitioning = true)
}

override def output: Seq[Attribute] = child.output

override def outputOrdering: Seq[SortOrder] = sortOrder
}

/**
* Optimized version of [[Sort]] that operates on binary data (implemented as part of
* Project Tungsten).
* Performs (external) sorting.
*
* @param global when true performs a global sort of all partitions by shuffling the data first
* if necessary.
* @param testSpillFrequency Method for configuring periodic spilling in unit tests. If set, will
* spill every `frequency` records.
*/

case class TungstenSort(
case class Sort(
sortOrder: Seq[SortOrder],
global: Boolean,
child: SparkPlan,
Expand Down Expand Up @@ -107,7 +61,7 @@ case class TungstenSort(
val dataSize = longMetric("dataSize")
val spillSize = longMetric("spillSize")

child.execute().mapPartitions { iter =>
child.execute().mapPartitionsInternal { iter =>
val ordering = newOrdering(sortOrder, childOutput)

// The comparator for comparing prefix
Expand Down Expand Up @@ -143,5 +97,4 @@ case class TungstenSort(
sortedIterator
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ class SparkPlanner(val sqlContext: SQLContext) extends SparkStrategies {
filterCondition.map(Filter(_, scan)).getOrElse(scan)
} else {
val scan = scanBuilder((projectSet ++ filterSet).toSeq)
TungstenProject(projectList, filterCondition.map(Filter(_, scan)).getOrElse(scan))
Project(projectList, filterCondition.map(Filter(_, scan)).getOrElse(scan))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -302,16 +302,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
object BasicOperators extends Strategy {
def numPartitions: Int = self.numPartitions

/**
* Picks an appropriate sort operator.
*
* @param global when true performs a global sort of all partitions by shuffling the data first
* if necessary.
*/
def getSortOperator(sortExprs: Seq[SortOrder], global: Boolean, child: SparkPlan): SparkPlan = {
execution.TungstenSort(sortExprs, global, child)
}

def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case r: RunnableCommand => ExecutedCommand(r) :: Nil

Expand Down Expand Up @@ -339,11 +329,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case logical.SortPartitions(sortExprs, child) =>
// This sort only sorts tuples within a partition. Its requiredDistribution will be
// an UnspecifiedDistribution.
getSortOperator(sortExprs, global = false, planLater(child)) :: Nil
execution.Sort(sortExprs, global = false, child = planLater(child)) :: Nil
case logical.Sort(sortExprs, global, child) =>
getSortOperator(sortExprs, global, planLater(child)):: Nil
execution.Sort(sortExprs, global, planLater(child)) :: Nil
case logical.Project(projectList, child) =>
execution.TungstenProject(projectList, planLater(child)) :: Nil
execution.Project(projectList, planLater(child)) :: Nil
case logical.Filter(condition, child) =>
execution.Filter(condition, planLater(child)) :: Nil
case e @ logical.Expand(_, _, child) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import org.apache.spark.util.random.PoissonSampler
import org.apache.spark.{HashPartitioner, SparkEnv}


case class TungstenProject(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode {
case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode {

override private[sql] lazy val metrics = Map(
"numRows" -> SQLMetrics.createLongMetric(sparkContext, "number of rows"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -343,7 +343,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
requestedColumns,
scanBuilder(requestedColumns, candidatePredicates, pushedFilters),
relation.relation)
execution.TungstenProject(
execution.Project(
projects, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan))
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql
import org.apache.spark.sql.catalyst.expressions.NamedExpression
import org.scalatest.Matchers._

import org.apache.spark.sql.execution.TungstenProject
import org.apache.spark.sql.execution.Project
import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -615,7 +615,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext {

def checkNumProjects(df: DataFrame, expectedNumProjects: Int): Unit = {
val projects = df.queryExecution.executedPlan.collect {
case tungstenProject: TungstenProject => tungstenProject
case tungstenProject: Project => tungstenProject
}
assert(projects.size === expectedNumProjects)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -365,7 +365,7 @@ class PlannerSuite extends SharedSQLContext {
)
val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan)
assertDistributionRequirementsAreSatisfied(outputPlan)
if (outputPlan.collect { case s: TungstenSort => true; case s: Sort => true }.isEmpty) {
if (outputPlan.collect { case s: Sort => true }.isEmpty) {
fail(s"Sort should have been added:\n$outputPlan")
}
}
Expand All @@ -381,7 +381,7 @@ class PlannerSuite extends SharedSQLContext {
)
val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan)
assertDistributionRequirementsAreSatisfied(outputPlan)
if (outputPlan.collect { case s: TungstenSort => true; case s: Sort => true }.nonEmpty) {
if (outputPlan.collect { case s: Sort => true }.nonEmpty) {
fail(s"No sorts should have been added:\n$outputPlan")
}
}
Expand All @@ -398,7 +398,7 @@ class PlannerSuite extends SharedSQLContext {
)
val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan)
assertDistributionRequirementsAreSatisfied(outputPlan)
if (outputPlan.collect { case s: TungstenSort => true; case s: Sort => true }.isEmpty) {
if (outputPlan.collect { case s: Sort => true }.isEmpty) {
fail(s"Sort should have been added:\n$outputPlan")
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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

import org.apache.spark.{InternalAccumulator, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder}
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.util.CompletionIterator
import org.apache.spark.util.collection.ExternalSorter


/**
* A reference sort implementation used to compare against our normal sort.
*/
case class ReferenceSort(
sortOrder: Seq[SortOrder],
global: Boolean,
child: SparkPlan)
extends UnaryNode {

override def requiredChildDistribution: Seq[Distribution] =
if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil

protected override def doExecute(): RDD[InternalRow] = attachTree(this, "sort") {
child.execute().mapPartitions( { iterator =>
val ordering = newOrdering(sortOrder, child.output)
val sorter = new ExternalSorter[InternalRow, Null, InternalRow](
TaskContext.get(), ordering = Some(ordering))
sorter.insertAll(iterator.map(r => (r.copy(), null)))
val baseIterator = sorter.iterator.map(_._1)
val context = TaskContext.get()
context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled)
context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled)
context.internalMetricsToAccumulators(
InternalAccumulator.PEAK_EXECUTION_MEMORY).add(sorter.peakMemoryUsedBytes)
CompletionIterator[InternalRow, Iterator[InternalRow]](baseIterator, sorter.stop())
}, preservesPartitioning = true)
}

override def output: Seq[Attribute] = child.output

override def outputOrdering: Seq[SortOrder] = sortOrder
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,9 +33,9 @@ class RowFormatConvertersSuite extends SparkPlanTest with SharedSQLContext {
case c: ConvertToSafe => c
}

private val outputsSafe = Sort(Nil, false, PhysicalRDD(Seq.empty, null, "name"))
private val outputsSafe = ReferenceSort(Nil, false, PhysicalRDD(Seq.empty, null, "name"))
assert(!outputsSafe.outputsUnsafeRows)
private val outputsUnsafe = TungstenSort(Nil, false, PhysicalRDD(Seq.empty, null, "name"))
private val outputsUnsafe = Sort(Nil, false, PhysicalRDD(Seq.empty, null, "name"))
assert(outputsUnsafe.outputsUnsafeRows)

test("planner should insert unsafe->safe conversions when required") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,22 @@

package org.apache.spark.sql.execution

import org.apache.spark.sql.Row
import scala.util.Random

import org.apache.spark.AccumulatorSuite
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
import org.apache.spark.sql.{RandomDataGenerator, Row}


/**
* Test sorting. Many of the test cases generate random data and compares the sorted result with one
* sorted by a reference implementation ([[ReferenceSort]]).
*/
class SortSuite extends SparkPlanTest with SharedSQLContext {
import testImplicits.localSeqToDataFrameHolder

// This test was originally added as an example of how to use [[SparkPlanTest]];
// it's not designed to be a comprehensive test of ExternalSort.
test("basic sorting using ExternalSort") {

val input = Seq(
Expand All @@ -36,14 +43,66 @@ class SortSuite extends SparkPlanTest with SharedSQLContext {

checkAnswer(
input.toDF("a", "b", "c"),
Sort('a.asc :: 'b.asc :: Nil, global = true, _: SparkPlan),
(child: SparkPlan) => Sort('a.asc :: 'b.asc :: Nil, global = true, child = child),
input.sortBy(t => (t._1, t._2)).map(Row.fromTuple),
sortAnswers = false)

checkAnswer(
input.toDF("a", "b", "c"),
Sort('b.asc :: 'a.asc :: Nil, global = true, _: SparkPlan),
(child: SparkPlan) => Sort('b.asc :: 'a.asc :: Nil, global = true, child = child),
input.sortBy(t => (t._2, t._1)).map(Row.fromTuple),
sortAnswers = false)
}

test("sort followed by limit") {
checkThatPlansAgree(
(1 to 100).map(v => Tuple1(v)).toDF("a"),
(child: SparkPlan) => Limit(10, Sort('a.asc :: Nil, global = true, child = child)),
(child: SparkPlan) => Limit(10, ReferenceSort('a.asc :: Nil, global = true, child)),
sortAnswers = false
)
}

test("sorting does not crash for large inputs") {
val sortOrder = 'a.asc :: Nil
val stringLength = 1024 * 1024 * 2
checkThatPlansAgree(
Seq(Tuple1("a" * stringLength), Tuple1("b" * stringLength)).toDF("a").repartition(1),
Sort(sortOrder, global = true, _: SparkPlan, testSpillFrequency = 1),
ReferenceSort(sortOrder, global = true, _: SparkPlan),
sortAnswers = false
)
}

test("sorting updates peak execution memory") {
AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "unsafe external sort") {
checkThatPlansAgree(
(1 to 100).map(v => Tuple1(v)).toDF("a"),
(child: SparkPlan) => Sort('a.asc :: Nil, global = true, child = child),
(child: SparkPlan) => ReferenceSort('a.asc :: Nil, global = true, child),
sortAnswers = false)
}
}

// Test sorting on different data types
for (
dataType <- DataTypeTestUtils.atomicTypes ++ Set(NullType);
nullable <- Seq(true, false);
sortOrder <- Seq('a.asc :: Nil, 'a.desc :: Nil);
randomDataGenerator <- RandomDataGenerator.forType(dataType, nullable)
) {
test(s"sorting on $dataType with nullable=$nullable, sortOrder=$sortOrder") {
val inputData = Seq.fill(1000)(randomDataGenerator())
val inputDf = sqlContext.createDataFrame(
sparkContext.parallelize(Random.shuffle(inputData).map(v => Row(v))),
StructType(StructField("a", dataType, nullable = true) :: Nil)
)
checkThatPlansAgree(
inputDf,
p => ConvertToSafe(Sort(sortOrder, global = true, p: SparkPlan, testSpillFrequency = 23)),
ReferenceSort(sortOrder, global = true, _: SparkPlan),
sortAnswers = false
)
}
}
}
Loading