From 83adb9d53245a7fec37f3cf7e2d6bb335e71b442 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 9 Jan 2014 23:28:55 -0500 Subject: [PATCH 01/34] add DataProperty Author: Yin Huai --- .../catalyst/execution/DataProperty.scala | 106 ++++++++++++ .../scala/catalyst/execution/Exchange.scala | 160 ++++++++++++++++++ .../catalyst/execution/SharkInstance.scala | 11 +- .../scala/catalyst/execution/SharkPlan.scala | 21 +++ .../scala/catalyst/execution/aggregates.scala | 13 +- .../catalyst/execution/basicOperators.scala | 40 +++-- src/main/scala/catalyst/execution/joins.scala | 17 +- .../execution/planningStrategies.scala | 8 +- src/main/scala/catalyst/plans/QueryPlan.scala | 30 +++- .../spark/rdd/CoGroupedLocallyRDD.scala | 85 ++++++++++ .../spark/rdd/SharkOrderedRDDFunctions.scala | 41 +++++ .../spark/rdd/SharkPairRDDFunctions.scala | 67 ++++++++ .../execution/DataPropertyTests.scala | 154 +++++++++++++++++ .../catalyst/execution/DslQueryTests1.scala | 74 ++++++++ 14 files changed, 798 insertions(+), 29 deletions(-) create mode 100644 src/main/scala/catalyst/execution/DataProperty.scala create mode 100644 src/main/scala/catalyst/execution/Exchange.scala create mode 100644 src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala create mode 100644 src/test/scala/catalyst/execution/DataPropertyTests.scala create mode 100644 src/test/scala/catalyst/execution/DslQueryTests1.scala diff --git a/src/main/scala/catalyst/execution/DataProperty.scala b/src/main/scala/catalyst/execution/DataProperty.scala new file mode 100644 index 0000000000000..b8179b97f832c --- /dev/null +++ b/src/main/scala/catalyst/execution/DataProperty.scala @@ -0,0 +1,106 @@ +package catalyst +package execution + +import catalyst.expressions.{SortOrder, Expression} + +trait DataProperty { + def expressions: Seq[Expression] + + /** + * If we need an [[catalyst.execution.Exchange]] to re-partition data for + * the given [[catalyst.execution.DataProperty]] other. + * @param other The given [[catalyst.execution.DataProperty]]. + *@return + */ + // TODO: We should also consider functional dependencies between expressions of + // two data properties. For example, if we have a GroupProperty(a) and a + // GroupProperty(a+1), we will not need an exchange to re-partition the data. + def needExchange(other: DataProperty): Boolean +} + +/** + * An implementation of [[catalyst.execution.DataProperty]] represents that + * the data property of a dataset is not specified. + * If it is used as a required data property for a physical operator + * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), + * it means that this operator does not require its input datasets to be + * organized in a certain way. + */ +case class NotSpecifiedProperty() extends DataProperty { + def expressions = Nil + + def needExchange(other: DataProperty): Boolean = { + other match { + case NotSpecifiedProperty() => false + case GroupProperty(groupingExpressions) => true + case SortProperty(_) => true + } + } +} + +/** + * An implementation of [[catalyst.execution.DataProperty]] represents that + * a dataset is grouped by groupingExpressions. + * @param groupingExpressions The expressions used to specify the way how rows should be grouped. + * If it is a Nil, the entire dataset is considered as a single group. + * In this case, a single reducer will be used. + */ +case class GroupProperty(groupingExpressions: Seq[Expression]) extends DataProperty { + override val expressions = groupingExpressions + + def needExchange(other: DataProperty): Boolean = { + other match { + case NotSpecifiedProperty() => false + // We do not need an Exchange operator if another GroupProperty only + // needs to group rows within a partition. + case g @ GroupProperty(otherExpressions) => { + if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true + } + // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, + // we need to use an Exchange operator to sort data with a + // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is + // a Nil, we will have a single partition. So, we do not need an Exchange operator + // to sort this single partition. + case SortProperty(otherExpressions) => if (expressions == Nil) false else true + } + } +} + +/** + * An implementation of [[catalyst.execution.DataProperty]] represents that + * a dataset is sorted by sortingExpressions. A SortProperty also implies that + * the dataset is grouped by sortingExpressions. + * @param sortingExpressions The expressions used to specify the way how rows should be sorted. + * sortingExpressions should not be empty. + */ +case class SortProperty(sortingExpressions: Seq[SortOrder]) extends DataProperty { + + { + if (sortingExpressions == Nil) { + throw new IllegalArgumentException("Sorting expressions of a SortProperty " + + "are not specified.") + } + } + + def expressions = sortingExpressions + + def needExchange(other: DataProperty): Boolean = { + other match { + case NotSpecifiedProperty() => false + // A SortProperty implies a GroupProperty. We do not need an Exchange operator + // if the GroupProperty only needs to group rows within a partition. + case g @ GroupProperty(otherExpressions) => { + if (expressions.map(expression => expression.child). + toSet.subsetOf(otherExpressions.toSet)) false else true + } + // We do not need an Exchange operator if another SortProperty only needs to + // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) + // or we do not need to sort again (cases satisfying + // expressions.startsWith(otherExpressions)). + case s @ SortProperty(otherExpressions) => { + if (otherExpressions.startsWith(expressions) || + expressions.startsWith(otherExpressions)) false else true + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala new file mode 100644 index 0000000000000..373684eb61ee8 --- /dev/null +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -0,0 +1,160 @@ +package catalyst +package execution + +import catalyst.rules.Rule +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ + +import org.apache.spark.{RangePartitioner, HashPartitioner} +import org.apache.spark.rdd.ShuffledRDD + +case class Exchange( + dataProperty: DataProperty, + child: SharkPlan, + numPartitions: Int = 8) extends UnaryNode { + + def output = child.output + + def execute() = attachTree(this , "execute") { + dataProperty match { + case NotSpecifiedProperty() => child.execute() + case g @ GroupProperty(groupingExpressions) => { + val rdd = child.execute().map { row => + (buildRow(groupingExpressions.toSeq.map(Evaluate(_, Vector(row)))), row) + } + val part = new HashPartitioner(numPartitions) + val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) + + shuffled.map(_._2) + } + case s @ SortProperty(sortingExpressions) => { + val directions = sortingExpressions.map(_.direction).toIndexedSeq + val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + + class SortKey(val keyValues: IndexedSeq[Any]) + extends Ordered[SortKey] + with Serializable { + def compare(other: SortKey): Int = { + var i = 0 + while (i < keyValues.size) { + val left = keyValues(i) + val right = other.keyValues(i) + val curDirection = directions(i) + val curDataType = dataTypes(i) + + logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") + // TODO: Use numeric here too? + val comparison = + if (left == null && right == null) { + 0 + } else if (left == null) { + if (curDirection == Ascending) -1 else 1 + } else if (right == null) { + if (curDirection == Ascending) 1 else -1 + } else if (curDataType == IntegerType) { + if (curDirection == Ascending) { + left.asInstanceOf[Int] compare right.asInstanceOf[Int] + } else { + right.asInstanceOf[Int] compare left.asInstanceOf[Int] + } + } else if (curDataType == DoubleType) { + if (curDirection == Ascending) { + left.asInstanceOf[Double] compare right.asInstanceOf[Double] + } else { + right.asInstanceOf[Double] compare left.asInstanceOf[Double] + } + } else if (curDataType == LongType) { + if (curDirection == Ascending) { + left.asInstanceOf[Long] compare right.asInstanceOf[Long] + } else { + right.asInstanceOf[Long] compare left.asInstanceOf[Long] + } + } else if (curDataType == StringType) { + if (curDirection == Ascending) { + left.asInstanceOf[String] compare right.asInstanceOf[String] + } else { + right.asInstanceOf[String] compare left.asInstanceOf[String] + } + } else { + sys.error(s"Comparison not yet implemented for: $curDataType") + } + + if (comparison != 0) return comparison + i += 1 + } + return 0 + } + } + + val rdd = child.execute().map { row => + val input = Vector(row) + val sortKey = new SortKey( + sortingExpressions.map(s => Evaluate(s.child, input)).toIndexedSeq) + + (sortKey, row) + } + val part = new RangePartitioner(numPartitions, rdd, ascending = true) + val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) + + shuffled.map(_._2) + } + } + } +} + +object AddExchange extends Rule[SharkPlan] { + def apply(plan: SharkPlan): SharkPlan = { + // TODO: determine the number of partitions. + // TODO: We need to consider the number of partitions to determine if we + // will add an Exchange operator. If a dataset only has a single partition, + // even if needExchange returns true, we do not need to shuffle the data again. + val numPartitions = 8 + plan.transformUp { + case aggregate @ Aggregate( + groupingExpressions, + aggregateExpressions, + child) => { + if (child.outputDataProperty.needExchange(aggregate.requiredDataProperty)) { + val exchange = new Exchange(aggregate.requiredDataProperty, child, numPartitions) + + Aggregate(groupingExpressions, aggregateExpressions, exchange)() + } else { + aggregate + } + } + case equiInnerJoin @ SparkEquiInnerJoin( + leftKeys, + rightKeys, + left, + right) => { + val newLeft = { + if (left.outputDataProperty.needExchange(equiInnerJoin.leftRequiredDataProperty)) { + new Exchange(equiInnerJoin.leftRequiredDataProperty, left, numPartitions) + } else { + left + } + } + + val newRight = { + if (right.outputDataProperty.needExchange(equiInnerJoin.rightRequiredDataProperty)) { + new Exchange(equiInnerJoin.rightRequiredDataProperty, right, numPartitions) + } else { + right + } + } + + SparkEquiInnerJoin(leftKeys, rightKeys, newLeft, newRight)() + } + case sort @ Sort(sortExprs, child) => { + if (child.outputDataProperty.needExchange(sort.requiredDataProperty)) { + val exchange = new Exchange(sort.requiredDataProperty, child, numPartitions) + + Sort(sortExprs, exchange)() + } else { + sort + } + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index c7d4eba52751f..1576d81cb6f56 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -82,7 +82,16 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = Batch("Prepare Expressions", Once, expressions.BindReferences) :: Nil + // TODO: [[catalyst.execution.AddExchange]] should be in a phase before + // PrepareForExecution. We may also need a phase of physical optimizations. + // In this phase, we can alter the data property in an Exchange operator to + // reduce the number of shuffling phases. Or, this physical optimization phase can be + // a part of QueryPlanner when a planner can accept a physical operator (a SharkPlan). + val batches = + Batch("Add exchange", Once, + AddExchange) :: + Batch("Prepare Expressions", Once, + expressions.BindReferences) :: Nil } class SharkSqlQuery(sql: String) extends SharkQuery { diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 6116d8397b090..8566354b57f65 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -4,10 +4,14 @@ package execution import org.apache.spark.rdd.RDD import catalyst.plans.QueryPlan +import scala.reflect.ClassTag abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => + def requiredDataProperties: Seq[DataProperty] + def outputDataProperty: DataProperty + /** * Runs this query returning the result as an RDD. */ @@ -23,12 +27,29 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { self: Product => + + def requiredDataProperty: DataProperty = NotSpecifiedProperty() + def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) + // TODO: We should get the output data properties of a leaf node from metadata. + def outputDataProperty: DataProperty = NotSpecifiedProperty() } trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => + + def requiredDataProperty: DataProperty = NotSpecifiedProperty() + def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) + + def outputDataProperty: DataProperty = child.outputDataProperty } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { self: Product => + + def leftRequiredDataProperty: DataProperty = NotSpecifiedProperty() + def rightRequiredDataProperty: DataProperty = NotSpecifiedProperty() + def requiredDataProperties: Seq[DataProperty] = + Seq(leftRequiredDataProperty, rightRequiredDataProperty) + + def outputDataProperty: DataProperty = NotSpecifiedProperty() } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 81ccaafd10d24..73920b0f9b243 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,16 +3,23 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ import org.apache.spark.SparkContext._ +import SharkPairRDDFunctions._ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SharkPlan) + (override val outputDataProperty: DataProperty = + GroupProperty(groupingExpressions)) extends UnaryNode { + override val requiredDataProperty: DataProperty = GroupProperty(groupingExpressions) + override def otherCopyArgs = outputDataProperty :: Nil + case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -86,9 +93,12 @@ case class Aggregate( def output = aggregateExpressions.map(_.toAttribute) def execute() = attachTree(this, "execute") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the groupingExpressions again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. val grouped = child.execute().map { row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) - }.groupByKey() + }.groupByKeyLocally() grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. @@ -142,6 +152,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan (@transient sc: SharkContext) extends UnaryNode { def output = aggregateExprs.map(_.toAttribute) override def otherCopyArgs = Seq(sc) + override val requiredDataProperty: DataProperty = GroupProperty(Nil) case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 75f5808618df7..a0b1874e03991 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -1,12 +1,12 @@ package catalyst package execution -import errors._ -import expressions._ -import types._ - -import org.apache.spark.SparkContext._ +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ +/* Implicits */ +import org.apache.spark.rdd.SharkOrderedRDDFunctions._ case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -45,12 +45,21 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { +case class Sort( + sortExprs: Seq[SortOrder], + child: SharkPlan) + (override val outputDataProperty: DataProperty = + SortProperty(sortExprs)) extends UnaryNode { val numPartitions = 1 // TODO: Set with input cardinality + override val requiredDataProperty: DataProperty = SortProperty(sortExprs) + override def otherCopyArgs = outputDataProperty :: Nil + private final val directions = sortExprs.map(_.direction).toIndexedSeq private final val dataTypes = sortExprs.map(_.dataType).toIndexedSeq + // TODO: This SortKey and the one in [[catalyst.execute.Exchange]] are the same. + // We should just define it only once. private class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { def compare(other: SortKey): Int = { var i = 0 @@ -81,12 +90,6 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { } else { right.asInstanceOf[Double] compare left.asInstanceOf[Double] } - } else if (curDataType == LongType) { - if (curDirection == Ascending) { - left.asInstanceOf[Long] compare right.asInstanceOf[Long] - } else { - right.asInstanceOf[Long] compare left.asInstanceOf[Long] - } } else if (curDataType == StringType) { if (curDirection == Ascending) { left.asInstanceOf[String] compare right.asInstanceOf[String] @@ -106,12 +109,16 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Don't include redundant expressions in both sortKey and row. def execute() = attachTree(this, "sort") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the sortKey again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. child.execute().map { row => val input = Vector(row) - val sortKey = new SortKey(sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) + val sortKey = new SortKey( + sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) (sortKey, row) - }.sortByKey(ascending = true, numPartitions).map(_._2) + }.sortByKeyLocally(ascending = true).map(_._2) } def output = child.output @@ -120,6 +127,9 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Rename: SchemaRDD case class LocalRelation(output: Seq[Attribute], data: Seq[IndexedSeq[Any]]) (@transient sc: SharkContext) extends LeafNode { - def execute() = sc.makeRDD(data.map(buildRow), 1) + + // Since LocalRelation is used for unit tests, set the defaultParallelism to 2 + // to make sure we can cover bugs appearing in a distributed environment. + def execute() = sc.makeRDD(data.map(buildRow), 2) } diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index fc5b745781642..bf81ff580222e 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -9,33 +9,38 @@ import errors._ import expressions._ import plans._ -/* Implicits */ -import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.SharkPairRDDFunctions._ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, right: SharkPlan) + (override val outputDataProperty: DataProperty = + GroupProperty(leftKeys)) extends BinaryNode { def output = left.output ++ right.output + override val leftRequiredDataProperty: DataProperty = GroupProperty(leftKeys) + override val rightRequiredDataProperty: DataProperty = GroupProperty(rightKeys) + override def otherCopyArgs = outputDataProperty :: Nil + def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute .map { row => + val leftWithKeys = left.execute.map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) logger.debug(s"Generated left join keys ($leftKeys) => ($joinKeys) given row $row") (joinKeys, row) } - val rightWithKeys = right.execute().map { row => + val rightWithKeys = right.execute.map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") (joinKeys, row) } // Do the join. - val joined = filterNulls(leftWithKeys).join(filterNulls(rightWithKeys)) + val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys)) // Drop join keys and merge input tuples. joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) } } @@ -63,7 +68,7 @@ case class BroadcastNestedLoopJoin( (@transient sc: SharkContext) extends BinaryNode { - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sc :: Nil def output = left.output ++ right.output diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2d00f3ccb7972..6b859c5e0f48d 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -88,7 +88,7 @@ trait PlanningStrategies { val rightKeys = joinKeys.map(_._2) val joinOp = execution.SparkEquiInnerJoin( - leftKeys, rightKeys, planLater(left), planLater(right)) + leftKeys, rightKeys, planLater(left), planLater(right))() // Make sure other conditions are met if present. if (otherPredicates.nonEmpty) { @@ -135,16 +135,16 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil // TODO: It is correct, but overkill to do a global sorting here. case logical.SortPartitions(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child)) :: Nil + execution.Aggregate(group, agg, planLater(child))() :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 3546416402f11..6f6212e0cea94 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -1,8 +1,10 @@ package catalyst package plans -import expressions.{Attribute, Expression} -import trees._ +import catalyst.expressions.{SortOrder, Attribute, Expression} +import catalyst.trees._ +import catalyst.execution.GroupProperty +import catalyst.execution.SortProperty abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -48,6 +50,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case other => other } + case g @ GroupProperty(groupingExpressions) => GroupProperty( + groupingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}) + case g @ SortProperty(sortingExpressions) => SortProperty( + sortingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -79,6 +89,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case other => other } + case g @ GroupProperty(groupingExpressions) => GroupProperty( + groupingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}) + case g @ SortProperty(sortingExpressions) => SortProperty( + sortingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -102,6 +120,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => e :: Nil case other => Nil } + case g @ GroupProperty(groupingExpressions) => groupingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } + case g @ SortProperty(sortingExpressions) => sortingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } case other => Nil }.toSeq } diff --git a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala new file mode 100644 index 0000000000000..7a7a6b61a05b0 --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala @@ -0,0 +1,85 @@ +package org.apache.spark.rdd + +import org.apache.spark._ +import org.apache.spark.util.AppendOnlyMap +import scala.collection.mutable.ArrayBuffer + +/** + * A RDD that cogroups its already co-partitioned parents. This RDD works the same as + * a [[org.apache.spark.rdd.CoGroupedRDD]] except that its parents should have the + * same number of partitions. Like a [[org.apache.spark.rdd.CoGroupedRDD]], + * for each key k in parent RDDs, the resulting RDD contains a tuple with the list of + * values for that key. + * + * @param rdds parent RDDs. + */ +class CoGroupedLocallyRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]]) + extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + + { + // Check if all parents have the same number of partitions. + // It is possible that a parent RDD does not preserve the partitioner, + // so we do not check if all of parent RDDs have the same partitioner. + if (!rdds.forall(rdd => rdd.partitions.size == rdds(0).partitions.size)) { + throw new IllegalArgumentException( + "All parent RDDs should have the same number of partitions.") + } + } + + // All dependencies of a CoGroupedLocallyRDD should be narrow dependencies. + override def getDependencies: Seq[Dependency[_]] = { + rdds.map { rdd: RDD[_ <: Product2[K, _]] => + logDebug("Adding one-to-one dependency with " + rdd) + new OneToOneDependency(rdd) + } + } + + override def getPartitions: Array[Partition] = { + val numPartitions = firstParent[(K, _)].partitions.size + val array = new Array[Partition](numPartitions) + for (i <- 0 until array.size) { + // Each CoGroupPartition will have a dependency per contributing RDD + array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => + new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) + }.toArray) + } + array + } + + // Take the first not None partitioner. It is possible that all parent partitioners + // are None. + override val partitioner = rdds.find(rdd => rdd.partitioner != None) match { + case Some(rdd) => rdd.partitioner + case None => None + } + + override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { + val split = s.asInstanceOf[CoGroupPartition] + val numRdds = split.deps.size + // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) + val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + + val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) + } + + val getSeq = (k: K) => { + map.changeValue(k, update) + } + + for ((dep, depNum) <- split.deps.zipWithIndex) dep match { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { + // Read them from the parent + rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv => + getSeq(kv._1)(depNum) += kv._2 + } + } + case _ => { + // We should not reach here. It is a sanity check. + throw new RuntimeException("A dependency of this CoGroupedLocallyRDD is not " + + "a narrow dependency.") + } + } + new InterruptibleIterator(context, map.iterator) + } +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala new file mode 100644 index 0000000000000..bd33b6bcd00eb --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala @@ -0,0 +1,41 @@ +package org.apache.spark.rdd + +import scala.reflect.ClassTag +import org.apache.spark.Logging + +import scala.language.implicitConversions + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs where the key is + * sortable through an implicit conversion. + * Import `org.apache.spark.SharkOrderedRDDFunctions._` at the top of your program to + * use these functions. They will work with any key type that has a `scala.math.Ordered` + * implementation. + */ +class SharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, + V: ClassTag, + P <: Product2[K, V] : ClassTag]( + self: RDD[P]) + extends Logging with Serializable { + + /** + * Sort data within a partition. This function will not introduce a + * shuffling operation. + */ + def sortByKeyLocally(ascending: Boolean = true): RDD[P] = { + self.mapPartitions(iter => { + val buf = iter.toArray + if (ascending) { + buf.sortWith((x, y) => x._1 < y._1).iterator + } else { + buf.sortWith((x, y) => x._1 > y._1).iterator + } + }, preservesPartitioning = true) + } +} + +object SharkOrderedRDDFunctions { + implicit def rddToSharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( + rdd: RDD[(K, V)]) = + new SharkOrderedRDDFunctions[K, V, (K, V)](rdd) +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala new file mode 100644 index 0000000000000..d26a43ac5720b --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -0,0 +1,67 @@ +package org.apache.spark.rdd + +import scala.reflect._ +import org.apache.spark._ +import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Aggregator +import org.apache.spark.SparkContext._ + +import scala.language.implicitConversions + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. + * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to use these functions. + */ +class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) + extends Logging + with Serializable { + + /** + * Cogroup corresponding partitions of `this` and `other`. These two RDDs should have + * the same number of partitions. Partitions of these two RDDs are cogrouped + * according to the indexes of partitions. If we have two RDDs and + * each of them has n partitions, we will cogroup the partition i from `this` + * with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + val cg = new CoGroupedLocallyRDD[K](Seq(self, other)) + val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag) + prfs.mapValues { case Seq(vs, ws) => + (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) + } + } + + /** + * Group the values for each key within a partition of the RDD into a single sequence. + * This function will not introduce a shuffling operation. + */ + def groupByKeyLocally(): RDD[(K, Seq[V])] = { + def createCombiner(v: V) = ArrayBuffer(v) + def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v + val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner _, mergeValue _, null) + val bufs = self.mapPartitionsWithContext((context, iter) => { + new InterruptibleIterator(context, aggregator.combineValuesByKey(iter)) + }, preservesPartitioning = true) + bufs.asInstanceOf[RDD[(K, Seq[V])]] + } + + /** + * Join corresponding partitions of `this` and `other`. + * If we have two RDDs and each of them has n partitions, + * we will join the partition i from `this` with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + cogroupLocally(other).flatMapValues { case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } +} + +object SharkPairRDDFunctions { + implicit def rddToSharkPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + new SharkPairRDDFunctions(rdd) +} + + diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala new file mode 100644 index 0000000000000..a459dc3e78aac --- /dev/null +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -0,0 +1,154 @@ +package catalyst.execution + +import catalyst.dsl._ + +import org.scalatest.FunSuite + +class DataPropertyTests extends FunSuite { + + protected def checkNeedExchange( + outputDataProperty: DataProperty, + inputDataProperty: DataProperty, + expected: Boolean) { + assert( + outputDataProperty.needExchange(inputDataProperty) === expected, + s""" + |== Output data property == + |$outputDataProperty + |== Input data property == + |$inputDataProperty + |== Expected result of needExchange == + |$expected + """.stripMargin) + } + + test("needExchange test: GroupProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + NotSpecifiedProperty(), + false) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + GroupProperty(Seq('b, 'c)), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + GroupProperty(Nil), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + GroupProperty(Nil), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + // Cases which need an exchange between two data properties. + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Seq('b, 'c)), + true) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Seq('d, 'e)), + true) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + GroupProperty(Nil), + true) + + checkNeedExchange( + GroupProperty(Seq('a, 'b, 'c)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + checkNeedExchange( + GroupProperty(Seq('b, 'c)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + // TODO: We should check functional dependencies + /* + checkNeedExchange( + GroupProperty(Seq('b)), + GroupProperty(Seq('b + 1)), + false) + */ + } + + test("needExchange test: SortProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + NotSpecifiedProperty(), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.asc)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('c, 'b, 'a)), + false) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('b, 'c, 'a, 'd)), + false) + + // Cases which need an exchange between two data properties. + // TODO: We can have an optimization to first sort the dataset + // by a.asc and then sort b, and c in a partition. This optimization + // should tradeoff the benefit of a less number of Exchange operators + // and the parallelism. + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('a.asc, 'b.desc, 'c.asc)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + SortProperty(Seq('b.asc, 'a.asc)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('a, 'b)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Seq('c, 'd)), + true) + + checkNeedExchange( + SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), + GroupProperty(Nil), + true) + } +} \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/DslQueryTests1.scala b/src/test/scala/catalyst/execution/DslQueryTests1.scala new file mode 100644 index 0000000000000..dbce6a9743ba4 --- /dev/null +++ b/src/test/scala/catalyst/execution/DslQueryTests1.scala @@ -0,0 +1,74 @@ +package catalyst +package execution + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import analysis._ +import expressions._ +import plans._ +import plans.logical.LogicalPlan +import types._ + +/* Implicits */ +import dsl._ + +class DslQueryTests1 extends FunSuite with BeforeAndAfterAll { + override def beforeAll() { + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests + // without restarting the JVM. + System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") + } + + import TestShark._ + import TestData._ + + test("inner join and then group by using the same column") { + val x = testData2.subquery('x) + val y = testData3.subquery('y) + checkAnswer( + x.join(y).where("x.a".attr === "y.a".attr) + .groupBy("x.a".attr)("x.a".attr, Count("x.b".attr)), + (1,2) :: + (2,2) :: Nil + ) + } + + /** + * Runs the plan and makes sure the answer matches the expected result. + * @param plan the query to be executed + * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. + */ + protected def checkAnswer(plan: LogicalPlan, expectedAnswer: Any): Unit = { + val convertedAnswer = expectedAnswer match { + case s: Seq[_] if s.isEmpty => s + case s: Seq[_] if s.head.isInstanceOf[Product] && + !s.head.isInstanceOf[Seq[_]] => s.map(_.asInstanceOf[Product].productIterator.toIndexedSeq) + case s: Seq[_] => s + case singleItem => Seq(Seq(singleItem)) + } + + val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty + def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer + val sharkAnswer = try plan.toRdd.collect().toSeq catch { + case e: Exception => + fail( + s""" + |Exception thrown while executing query: + |$plan + |== Physical Plan == + |${plan.executedPlan} + |== Exception == + |$e + """.stripMargin) + } + println( + s""" + |Logical plan: + |$plan + |== Physical Plan == + |${plan.executedPlan} + """.stripMargin) + assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) + } +} \ No newline at end of file From ec906200862e8a3b1a9c7f07e86707f90e81d2be Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:20:35 -0800 Subject: [PATCH 02/34] Support for Sets as arguments to TreeNode classes. --- src/main/scala/catalyst/plans/QueryPlan.scala | 7 +++++-- src/main/scala/catalyst/trees/TreeNode.scala | 9 ++++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 6f6212e0cea94..6f91f3ff795ee 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -46,7 +46,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy val newArgs = productIterator.map { case e: Expression => transformExpressionDown(e) case Some(e: Expression) => Some(transformExpressionDown(e)) - case seq: Seq[_] => seq.map { + case m: Map[_,_] => m + case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionDown(e) case other => other } @@ -85,7 +86,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy val newArgs = productIterator.map { case e: Expression => transformExpressionUp(e) case Some(e: Expression) => Some(transformExpressionUp(e)) - case seq: Seq[_] => seq.map { + case m: Map[_,_] => m + case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionUp(e) case other => other } @@ -125,6 +127,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case other => Nil } case g @ SortProperty(sortingExpressions) => sortingExpressions.flatMap { + case seq: Traversable[_] => seq.flatMap { case e: Expression => e :: Nil case other => Nil } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index a4af9b84c8fef..5f0bb342edcdf 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -125,7 +125,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } else { arg } - case args: Seq[_] => args.map { + case m: Map[_,_] => m + case args: Traversable[_] => args.map { case arg: TreeNode[_] if (children contains arg) => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { @@ -168,7 +169,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } else { arg } - case args: Seq[_] => args.map { + case m: Map[_,_] => m + case args: Traversable[_] => args.map { case arg: TreeNode[_] if (children contains arg) => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { @@ -225,7 +227,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def argString: String = productIterator.flatMap { case tn: TreeNode[_] if children contains tn => Nil case tn: TreeNode[_] if tn.toString contains "\n" => s"(${tn.simpleString})" :: Nil - case seq: Seq[_] => seq.mkString("{", ",", "}") :: Nil + case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil + case seq: Set[_] => seq.mkString("{", ",", "}") :: Nil case other => other :: Nil }.mkString(", ") From 037a2ed8afb229718433cc8d075f6d05084e5a1a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:20:53 -0800 Subject: [PATCH 03/34] Better exception when unbound attributes make it to evaluation. --- src/main/scala/catalyst/expressions/Evaluate.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index 4943bdb395444..b563041590c02 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -226,6 +226,7 @@ object Evaluate extends Logging { case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) + case a: Attribute => throw new OptimizationException(a, "Unable to evaluate unbound reference without access to the input schema.") case other => throw new OptimizationException(other, "evaluation not implemented") } From dfce4268b075622c3553c349c33e99a4b0235d21 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:21:22 -0800 Subject: [PATCH 04/34] Add mapChildren and withNewChildren methods to TreeNode. --- src/main/scala/catalyst/trees/TreeNode.scala | 46 ++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index 5f0bb342edcdf..dd2085950fc3d 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -83,6 +83,52 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { ret } + /** + * Returns a copy of this node where `f` has been applied to all the nodes children. + */ + def mapChildren(f: BaseType => BaseType): this.type = { + var changed = false + val newArgs = productIterator.map { + case arg: TreeNode[_] if (children contains arg) => + val newChild = f(arg.asInstanceOf[BaseType]) + if(newChild fastEquals arg) + arg + else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + if(changed) makeCopy(newArgs) else this + } + + /** + * Returns a copy of this node with the children replaced. + * TODO: Validate somewhere (in debug mode?) that children are ordered correctly. + */ + def withNewChildren(newChildren: Seq[BaseType]): this.type = { + assert(newChildren.size == children.size, "Incorrect number of children") + var changed = false + val remainingNewChildren = newChildren.toBuffer + val remainingOldChildren = children.toBuffer + val newArgs = productIterator.map { + case arg: TreeNode[_] if (children contains arg) => + val newChild = remainingNewChildren.remove(0) + val oldChild = remainingOldChildren.remove(0) + if(newChild fastEquals oldChild) + oldChild + else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + + if(changed) makeCopy(newArgs) else this + } + /** * Returns a copy of this node where `rule` has been recursively applied to the tree. * When `rule` does not apply to a given node it is left unchanged. From a1ef62e3d24a9d3cac072e0ab53a0017cc185c94 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:21:45 -0800 Subject: [PATCH 05/34] Print the executed plan in SharkQuery toString. --- src/main/scala/catalyst/execution/SharkInstance.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 1576d81cb6f56..122f2031c8def 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -144,7 +144,7 @@ abstract class SharkInstance extends Logging { s"""== Logical Plan == |${stringOrError(analyzed)} |== Physical Plan == - |${stringOrError(sharkPlan)} + |${stringOrError(executedPlan)} """.stripMargin.trim } From 60f3a9a0b2026ff159186e8d9921e4f2d9700cc6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:22:46 -0800 Subject: [PATCH 06/34] More aggregate functions out of the aggregate class to make things more readable. --- .../scala/catalyst/execution/aggregates.scala | 138 +++++++++--------- 1 file changed, 70 insertions(+), 68 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 73920b0f9b243..7c0c8fa4858fe 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -20,75 +20,7 @@ case class Aggregate( override val requiredDataProperty: DataProperty = GroupProperty(groupingExpressions) override def otherCopyArgs = outputDataProperty :: Nil - case class AverageFunction(expr: Expression, base: AggregateExpression) - extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var count: Long = _ - var sum: Long = _ - - def result: Any = sum.toDouble / count.toDouble - - def apply(input: Seq[Row]): Unit = { - count += 1 - // TODO: Support all types here... - sum += Evaluate(expr, input).asInstanceOf[Int] - } - } - - case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var count: Int = _ - - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { - count += 1 - } - } - - def result: Any = count - } - - case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) - - def apply(input: Seq[Row]): Unit = - sum = Evaluate(Add(Literal(sum), expr), input) - - def result: Any = sum - } - - case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) - extends AggregateFunction { - - def this() = this(null, null) // Required for serialization. - - val seen = new scala.collection.mutable.HashSet[Any]() - - def apply(input: Seq[Row]): Unit = { - val evaluatedExpr = expr.map(Evaluate(_, input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) - seen += evaluatedExpr - } - - def result: Any = seen.size - } - - case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { - def this() = this(null, null) // Required for serialization. - - var result: Any = null - - def apply(input: Seq[Row]): Unit = { - if (result == null) - result = Evaluate(expr, input) - } - } def output = aggregateExpressions.map(_.toAttribute) @@ -143,6 +75,76 @@ case class Aggregate( } } +case class AverageFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + var count: Long = _ + var sum: Long = _ + + def result: Any = sum.toDouble / count.toDouble + + def apply(input: Seq[Row]): Unit = { + count += 1 + // TODO: Support all types here... + sum += Evaluate(expr, input).asInstanceOf[Int] + } +} + +case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var count: Int = _ + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = expr.map(Evaluate(_, input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { + count += 1 + } + } + + def result: Any = count +} + +case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var sum = Evaluate(Cast(Literal(0), expr.dataType), Nil) + + def apply(input: Seq[Row]): Unit = + sum = Evaluate(Add(Literal(sum), expr), input) + + def result: Any = sum +} + +case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new scala.collection.mutable.HashSet[Any]() + + def apply(input: Seq[Row]): Unit = { + val evaluatedExpr = expr.map(Evaluate(_, input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) + seen += evaluatedExpr + } + + def result: Any = seen.size +} + +case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var result: Any = null + + def apply(input: Seq[Row]): Unit = { + if (result == null) + result = Evaluate(expr, input) + } +} + /** * Uses spark Accumulators to perform global aggregation. * From 098dfc42642438e9c97f16fcc072fad765cbc161 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 17:24:14 -0800 Subject: [PATCH 07/34] Implement Long sorting again. --- src/main/scala/catalyst/execution/basicOperators.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index a0b1874e03991..746c81090aeaf 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -90,6 +90,12 @@ case class Sort( } else { right.asInstanceOf[Double] compare left.asInstanceOf[Double] } + } else if (curDataType == LongType) { + if (curDirection == Ascending) { + left.asInstanceOf[Long] compare right.asInstanceOf[Long] + } else { + right.asInstanceOf[Long] compare left.asInstanceOf[Long] + } } else if (curDataType == StringType) { if (curDirection == Ascending) { left.asInstanceOf[String] compare right.asInstanceOf[String] From 2abb0bc0042ccafaff50c045f23b46be6ae76f89 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 18:53:01 -0800 Subject: [PATCH 08/34] better debug messages, use exists. --- src/main/scala/catalyst/execution/joins.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index bf81ff580222e..192149292f1aa 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -29,13 +29,13 @@ case class SparkEquiInnerJoin( def execute() = attachTree(this, "execute") { val leftWithKeys = left.execute.map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) - logger.debug(s"Generated left join keys ($leftKeys) => ($joinKeys) given row $row") + logger.debug(s"Generated left join keys [${leftKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } val rightWithKeys = right.execute.map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) - logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") + logger.debug(s"Generated right join keys [${rightKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } @@ -51,7 +51,7 @@ case class SparkEquiInnerJoin( */ protected def filterNulls(rdd: RDD[(Seq[Any], Row)]) = rdd.filter { - case (key: Seq[_], _) => !key.map(_ == null).reduceLeft(_ || _) + case (key: Seq[_], _) => !key.exists(_ == null) } } From fb84ae40f85a62b4f01ca31c6c0a945223df68ba Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Jan 2014 18:53:41 -0800 Subject: [PATCH 09/34] Refactor DataProperty into Distribution. --- .../catalyst/execution/DataProperty.scala | 106 --------- .../scala/catalyst/execution/Exchange.scala | 89 +++---- .../catalyst/execution/SharkInstance.scala | 5 - .../scala/catalyst/execution/SharkPlan.scala | 26 +-- .../scala/catalyst/execution/aggregates.scala | 10 +- .../catalyst/execution/basicOperators.scala | 11 +- .../catalyst/execution/distribution.scala | 88 +++++++ src/main/scala/catalyst/execution/joins.scala | 13 +- .../execution/planningStrategies.scala | 11 +- src/main/scala/catalyst/plans/QueryPlan.scala | 27 --- .../execution/DataPropertyTests.scala | 219 +++++++++--------- .../execution/HiveComparisonTest.scala | 3 +- 12 files changed, 245 insertions(+), 363 deletions(-) delete mode 100644 src/main/scala/catalyst/execution/DataProperty.scala create mode 100644 src/main/scala/catalyst/execution/distribution.scala diff --git a/src/main/scala/catalyst/execution/DataProperty.scala b/src/main/scala/catalyst/execution/DataProperty.scala deleted file mode 100644 index b8179b97f832c..0000000000000 --- a/src/main/scala/catalyst/execution/DataProperty.scala +++ /dev/null @@ -1,106 +0,0 @@ -package catalyst -package execution - -import catalyst.expressions.{SortOrder, Expression} - -trait DataProperty { - def expressions: Seq[Expression] - - /** - * If we need an [[catalyst.execution.Exchange]] to re-partition data for - * the given [[catalyst.execution.DataProperty]] other. - * @param other The given [[catalyst.execution.DataProperty]]. - *@return - */ - // TODO: We should also consider functional dependencies between expressions of - // two data properties. For example, if we have a GroupProperty(a) and a - // GroupProperty(a+1), we will not need an exchange to re-partition the data. - def needExchange(other: DataProperty): Boolean -} - -/** - * An implementation of [[catalyst.execution.DataProperty]] represents that - * the data property of a dataset is not specified. - * If it is used as a required data property for a physical operator - * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), - * it means that this operator does not require its input datasets to be - * organized in a certain way. - */ -case class NotSpecifiedProperty() extends DataProperty { - def expressions = Nil - - def needExchange(other: DataProperty): Boolean = { - other match { - case NotSpecifiedProperty() => false - case GroupProperty(groupingExpressions) => true - case SortProperty(_) => true - } - } -} - -/** - * An implementation of [[catalyst.execution.DataProperty]] represents that - * a dataset is grouped by groupingExpressions. - * @param groupingExpressions The expressions used to specify the way how rows should be grouped. - * If it is a Nil, the entire dataset is considered as a single group. - * In this case, a single reducer will be used. - */ -case class GroupProperty(groupingExpressions: Seq[Expression]) extends DataProperty { - override val expressions = groupingExpressions - - def needExchange(other: DataProperty): Boolean = { - other match { - case NotSpecifiedProperty() => false - // We do not need an Exchange operator if another GroupProperty only - // needs to group rows within a partition. - case g @ GroupProperty(otherExpressions) => { - if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true - } - // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, - // we need to use an Exchange operator to sort data with a - // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is - // a Nil, we will have a single partition. So, we do not need an Exchange operator - // to sort this single partition. - case SortProperty(otherExpressions) => if (expressions == Nil) false else true - } - } -} - -/** - * An implementation of [[catalyst.execution.DataProperty]] represents that - * a dataset is sorted by sortingExpressions. A SortProperty also implies that - * the dataset is grouped by sortingExpressions. - * @param sortingExpressions The expressions used to specify the way how rows should be sorted. - * sortingExpressions should not be empty. - */ -case class SortProperty(sortingExpressions: Seq[SortOrder]) extends DataProperty { - - { - if (sortingExpressions == Nil) { - throw new IllegalArgumentException("Sorting expressions of a SortProperty " + - "are not specified.") - } - } - - def expressions = sortingExpressions - - def needExchange(other: DataProperty): Boolean = { - other match { - case NotSpecifiedProperty() => false - // A SortProperty implies a GroupProperty. We do not need an Exchange operator - // if the GroupProperty only needs to group rows within a partition. - case g @ GroupProperty(otherExpressions) => { - if (expressions.map(expression => expression.child). - toSet.subsetOf(otherExpressions.toSet)) false else true - } - // We do not need an Exchange operator if another SortProperty only needs to - // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) - // or we do not need to sort again (cases satisfying - // expressions.startsWith(otherExpressions)). - case s @ SortProperty(otherExpressions) => { - if (otherExpressions.startsWith(expressions) || - expressions.startsWith(otherExpressions)) false else true - } - } - } -} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 373684eb61ee8..14c4019cdc30e 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -9,29 +9,29 @@ import catalyst.types._ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD -case class Exchange( - dataProperty: DataProperty, - child: SharkPlan, - numPartitions: Int = 8) extends UnaryNode { +case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: SharkPlan) + extends UnaryNode { + override def outputPartitioning = newPartitioning def output = child.output def execute() = attachTree(this , "execute") { - dataProperty match { - case NotSpecifiedProperty() => child.execute() - case g @ GroupProperty(groupingExpressions) => { + newPartitioning match { + case ClusteredDistribution(expressions) => + // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => - (buildRow(groupingExpressions.toSeq.map(Evaluate(_, Vector(row)))), row) + (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) } val part = new HashPartitioner(numPartitions) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - } - case s @ SortProperty(sortingExpressions) => { + + case OrderedDistribution(sortingExpressions) => val directions = sortingExpressions.map(_.direction).toIndexedSeq val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + // TODO: MOVE THIS! class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { @@ -98,63 +98,26 @@ case class Exchange( val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) shuffled.map(_._2) - } + case UnknownDistribution => + logger.warn("Worthless repartitioning requested.") + child.execute() } } } object AddExchange extends Rule[SharkPlan] { - def apply(plan: SharkPlan): SharkPlan = { - // TODO: determine the number of partitions. - // TODO: We need to consider the number of partitions to determine if we - // will add an Exchange operator. If a dataset only has a single partition, - // even if needExchange returns true, we do not need to shuffle the data again. - val numPartitions = 8 - plan.transformUp { - case aggregate @ Aggregate( - groupingExpressions, - aggregateExpressions, - child) => { - if (child.outputDataProperty.needExchange(aggregate.requiredDataProperty)) { - val exchange = new Exchange(aggregate.requiredDataProperty, child, numPartitions) - - Aggregate(groupingExpressions, aggregateExpressions, exchange)() - } else { - aggregate - } - } - case equiInnerJoin @ SparkEquiInnerJoin( - leftKeys, - rightKeys, - left, - right) => { - val newLeft = { - if (left.outputDataProperty.needExchange(equiInnerJoin.leftRequiredDataProperty)) { - new Exchange(equiInnerJoin.leftRequiredDataProperty, left, numPartitions) - } else { - left - } - } - - val newRight = { - if (right.outputDataProperty.needExchange(equiInnerJoin.rightRequiredDataProperty)) { - new Exchange(equiInnerJoin.rightRequiredDataProperty, right, numPartitions) - } else { - right - } - } - - SparkEquiInnerJoin(leftKeys, rightKeys, newLeft, newRight)() - } - case sort @ Sort(sortExprs, child) => { - if (child.outputDataProperty.needExchange(sort.requiredDataProperty)) { - val exchange = new Exchange(sort.requiredDataProperty, child, numPartitions) - - Sort(sortExprs, exchange)() - } else { - sort - } - } - } + // TODO: determine the number of partitions. + // TODO: We need to consider the number of partitions to determine if we + // will add an Exchange operator. If a dataset only has a single partition, + // even if needExchange returns true, we do not need to shuffle the data again. + val numPartitions = 8 + + def apply(plan: SharkPlan): SharkPlan = plan.transformUp { + case operator: SharkPlan => + operator.withNewChildren(operator.requiredChildPartitioning.zip(operator.children).map { + case (required, child) if !child.outputPartitioning.satisfies(required) => + Exchange(required, numPartitions, child) + case (_, child) => child + }) } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index 122f2031c8def..86ceb89ba0d49 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -82,11 +82,6 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - // TODO: [[catalyst.execution.AddExchange]] should be in a phase before - // PrepareForExecution. We may also need a phase of physical optimizations. - // In this phase, we can alter the data property in an Exchange operator to - // reduce the number of shuffling phases. Or, this physical optimization phase can be - // a part of QueryPlanner when a planner can accept a physical operator (a SharkPlan). val batches = Batch("Add exchange", Once, AddExchange) :: diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 8566354b57f65..fc46b1494d8a5 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -3,14 +3,15 @@ package execution import org.apache.spark.rdd.RDD -import catalyst.plans.QueryPlan -import scala.reflect.ClassTag +import plans.QueryPlan abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => - def requiredDataProperties: Seq[DataProperty] - def outputDataProperty: DataProperty + /** Specifies how data is partitioned across different nodes in the cluster. */ + def outputPartitioning: Distribution = UnknownDistribution + /** Specifies any partition requirements on the input data for this operator. */ + def requiredChildPartitioning: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) /** * Runs this query returning the result as an RDD. @@ -27,29 +28,12 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { self: Product => - - def requiredDataProperty: DataProperty = NotSpecifiedProperty() - def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) - // TODO: We should get the output data properties of a leaf node from metadata. - def outputDataProperty: DataProperty = NotSpecifiedProperty() } trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => - - def requiredDataProperty: DataProperty = NotSpecifiedProperty() - def requiredDataProperties: Seq[DataProperty] = Seq(requiredDataProperty) - - def outputDataProperty: DataProperty = child.outputDataProperty } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { self: Product => - - def leftRequiredDataProperty: DataProperty = NotSpecifiedProperty() - def rightRequiredDataProperty: DataProperty = NotSpecifiedProperty() - def requiredDataProperties: Seq[DataProperty] = - Seq(leftRequiredDataProperty, rightRequiredDataProperty) - - def outputDataProperty: DataProperty = NotSpecifiedProperty() } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 7c0c8fa4858fe..89fa0912bb147 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -13,14 +13,10 @@ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SharkPlan) - (override val outputDataProperty: DataProperty = - GroupProperty(groupingExpressions)) extends UnaryNode { - override val requiredDataProperty: DataProperty = GroupProperty(groupingExpressions) - override def otherCopyArgs = outputDataProperty :: Nil - - + val requiredPartitioning = ClusteredDistribution(groupingExpressions) + override def requiredChildPartitioning = Seq(requiredPartitioning, requiredPartitioning) def output = aggregateExpressions.map(_.toAttribute) @@ -46,6 +42,7 @@ case class Aggregate( val remainingAttributes = impl.collect { case a: Attribute => a } // If any references exist that are not inside agg functions then the must be grouping exprs // in this case we must rebind them to the grouping tuple. + // TODO: Is this right still? Do we need this? if (remainingAttributes.nonEmpty) { val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } @@ -154,7 +151,6 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan (@transient sc: SharkContext) extends UnaryNode { def output = aggregateExprs.map(_.toAttribute) override def otherCopyArgs = Seq(sc) - override val requiredDataProperty: DataProperty = GroupProperty(Nil) case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 746c81090aeaf..4b253247994c3 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -45,16 +45,7 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class Sort( - sortExprs: Seq[SortOrder], - child: SharkPlan) - (override val outputDataProperty: DataProperty = - SortProperty(sortExprs)) extends UnaryNode { - val numPartitions = 1 // TODO: Set with input cardinality - - override val requiredDataProperty: DataProperty = SortProperty(sortExprs) - override def otherCopyArgs = outputDataProperty :: Nil - +case class SortPartitions(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { private final val directions = sortExprs.map(_.direction).toIndexedSeq private final val dataTypes = sortExprs.map(_.dataType).toIndexedSeq diff --git a/src/main/scala/catalyst/execution/distribution.scala b/src/main/scala/catalyst/execution/distribution.scala new file mode 100644 index 0000000000000..4ffd5a3f61cf7 --- /dev/null +++ b/src/main/scala/catalyst/execution/distribution.scala @@ -0,0 +1,88 @@ +package catalyst +package execution + +import expressions.{Expression, SortOrder} +import types._ + +/** + * Specifies how tuples that share common expressions will be distributed. Distribution can + * be used to refer to two distinct physical properties: + * - Inter-node partitioning of data: In this case the distribution describes how tuples are + * partitioned across physical machines in a cluster. Knowing this property allows some + * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. + * - Intra-partition ordering of data: In this case the distribution describes guarantees made + * about how tuples are distributed within a single partition. + * + * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping + * actually does matter, and thus our subset checking is probably not sufficient + * to ensure correct colocation for joins.? + * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort + * aggregation can be done. Maybe we need two checks? Separate ordering from + * partitioning? + */ +abstract sealed class Distribution extends Expression { + self: Product => + + /** + * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all + * guarantees mandated by the `required` distribution. + */ + def satisfies(required: Distribution): Boolean +} + +/** + * Represents a distribution where no promises are made about co-location of data. + */ +case object UnknownDistribution extends Distribution with trees.LeafNode[Expression] { + def references = Set.empty + def nullable = false + def dataType = NullType + + def satisfies(required: Distribution): Boolean = required == UnknownDistribution +} + +/** + * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will + * be co-located. Based on the context, this can mean such tuples are either co-located in the same + * partition or they will be contiguous within a single partition. + */ +case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { + def children = clustering.toSeq + def references = clustering.flatMap(_.references).toSet + def nullable = false + def dataType = NullType + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + // No clustering expressions means only one partition. + case _ if clustering.isEmpty => true + case ClusteredDistribution(requiredClustering) => + clustering.toSet.subsetOf(requiredClustering.toSet) + case _ => false + } +} + +/** + * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This + * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that + * tuples that share the same value for the ordering expressions are contiguous and will never be + * split across partitions. + */ +case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { + def children = ordering + def references = ordering.flatMap(_.references).toSet + def nullable = false + def dataType = NullType + + def clustering = ordering.map(_.child).toSet + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case OrderedDistribution(requiredOrdering) => + val minSize = Seq(requiredOrdering.size, ordering.size).min + requiredOrdering.take(minSize) == ordering.take(minSize) + case ClusteredDistribution(requiredClustering) => + clustering.subsetOf(requiredClustering.toSet) + case _ => false + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 192149292f1aa..6f811f20a737d 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -15,16 +15,13 @@ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, - right: SharkPlan) - (override val outputDataProperty: DataProperty = - GroupProperty(leftKeys)) - extends BinaryNode { + right: SharkPlan) extends BinaryNode { - def output = left.output ++ right.output - override val leftRequiredDataProperty: DataProperty = GroupProperty(leftKeys) - override val rightRequiredDataProperty: DataProperty = GroupProperty(rightKeys) - override def otherCopyArgs = outputDataProperty :: Nil + override def requiredChildPartitioning = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + def output = left.output ++ right.output def execute() = attachTree(this, "execute") { val leftWithKeys = left.execute.map { row => diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 6b859c5e0f48d..bcce25697ac84 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -88,7 +88,7 @@ trait PlanningStrategies { val rightKeys = joinKeys.map(_._2) val joinOp = execution.SparkEquiInnerJoin( - leftKeys, rightKeys, planLater(left), planLater(right))() + leftKeys, rightKeys, planLater(left), planLater(right)) // Make sure other conditions are met if present. if (otherPredicates.nonEmpty) { @@ -135,16 +135,17 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, planLater(child))() :: Nil - // TODO: It is correct, but overkill to do a global sorting here. + // First repartition then sort locally. + execution.SortPartitions(sortExprs, + execution.Exchange(OrderedDistribution(sortExprs), 8, planLater(child))) :: Nil case logical.SortPartitions(sortExprs, child) => - execution.Sort(sortExprs, planLater(child))() :: Nil + execution.SortPartitions(sortExprs, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child))() :: Nil + execution.Aggregate(group, agg, planLater(child)) :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 6f91f3ff795ee..8a2ed4201602d 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -3,8 +3,6 @@ package plans import catalyst.expressions.{SortOrder, Attribute, Expression} import catalyst.trees._ -import catalyst.execution.GroupProperty -import catalyst.execution.SortProperty abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -51,14 +49,6 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case other => other } - case g @ GroupProperty(groupingExpressions) => GroupProperty( - groupingExpressions.map { - case e: Expression => transformExpressionDown(e) - case other => other}) - case g @ SortProperty(sortingExpressions) => SortProperty( - sortingExpressions.map { - case e: Expression => transformExpressionDown(e) - case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -91,14 +81,6 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case other => other } - case g @ GroupProperty(groupingExpressions) => GroupProperty( - groupingExpressions.map { - case e: Expression => transformExpressionUp(e) - case other => other}) - case g @ SortProperty(sortingExpressions) => SortProperty( - sortingExpressions.map { - case e: Expression => transformExpressionUp(e) - case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -118,15 +100,6 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy productIterator.flatMap { case e: Expression => e :: Nil case Some(e: Expression) => e :: Nil - case seq: Seq[_] => seq.flatMap { - case e: Expression => e :: Nil - case other => Nil - } - case g @ GroupProperty(groupingExpressions) => groupingExpressions.flatMap { - case e: Expression => e :: Nil - case other => Nil - } - case g @ SortProperty(sortingExpressions) => sortingExpressions.flatMap { case seq: Traversable[_] => seq.flatMap { case e: Expression => e :: Nil case other => Nil diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala index a459dc3e78aac..933897cff3922 100644 --- a/src/test/scala/catalyst/execution/DataPropertyTests.scala +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -4,151 +4,150 @@ import catalyst.dsl._ import org.scalatest.FunSuite -class DataPropertyTests extends FunSuite { - - protected def checkNeedExchange( - outputDataProperty: DataProperty, - inputDataProperty: DataProperty, - expected: Boolean) { - assert( - outputDataProperty.needExchange(inputDataProperty) === expected, - s""" - |== Output data property == - |$outputDataProperty - |== Input data property == - |$inputDataProperty - |== Expected result of needExchange == - |$expected - """.stripMargin) +class DistributionTests extends FunSuite { + protected def checkSatisfied( + inputDistribution: Distribution, + requiredDistribution: Distribution, + satisfied: Boolean) { + if(inputDistribution.satisfies(requiredDistribution) != satisfied) + fail( + s""" + |== Input Distribution == + |$inputDistribution + |== Required Distribution == + |$requiredDistribution + |== Does input distribution satisfy requirements? == + |Expected $satisfied got ${inputDistribution.satisfies(requiredDistribution)} + """.stripMargin) } - test("needExchange test: GroupProperty is the output DataProperty") { + test("needExchange test: ClusteredDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - NotSpecifiedProperty(), - false) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + UnknownDistribution, + true) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - GroupProperty(Seq('b, 'c)), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + ClusteredDistribution(Seq('b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - GroupProperty(Nil), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + ClusteredDistribution(Nil), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - GroupProperty(Nil), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - false) + checkSatisfied( + ClusteredDistribution(Nil), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + true) // Cases which need an exchange between two data properties. - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Seq('b, 'c)), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('b, 'c)), + false) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Seq('d, 'e)), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('d, 'e)), + false) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - GroupProperty(Nil), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Nil), + false) - checkNeedExchange( - GroupProperty(Seq('a, 'b, 'c)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - true) + checkSatisfied( + ClusteredDistribution(Seq('a, 'b, 'c)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + false) - checkNeedExchange( - GroupProperty(Seq('b, 'c)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - true) + checkSatisfied( + ClusteredDistribution(Seq('b, 'c)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + false) // TODO: We should check functional dependencies /* - checkNeedExchange( - GroupProperty(Seq('b)), - GroupProperty(Seq('b + 1)), - false) + checkSatisfied( + ClusteredDistribution(Seq('b)), + ClusteredDistribution(Seq('b + 1)), + true) */ } - test("needExchange test: SortProperty is the output DataProperty") { + test("needExchange test: OrderedDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - NotSpecifiedProperty(), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + UnknownDistribution, + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.asc)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('a, 'b, 'c)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('c, 'b, 'a)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('c, 'b, 'a)), + true) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('b, 'c, 'a, 'd)), - false) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('b, 'c, 'a, 'd)), + true) // Cases which need an exchange between two data properties. // TODO: We can have an optimization to first sort the dataset // by a.asc and then sort b, and c in a partition. This optimization // should tradeoff the benefit of a less number of Exchange operators // and the parallelism. - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('a.asc, 'b.desc, 'c.asc)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - SortProperty(Seq('b.asc, 'a.asc)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('b.asc, 'a.asc)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('a, 'b)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('a, 'b)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Seq('c, 'd)), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Seq('c, 'd)), + false) - checkNeedExchange( - SortProperty(Seq('a.asc, 'b.asc, 'c.asc)), - GroupProperty(Nil), - true) + checkSatisfied( + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + ClusteredDistribution(Nil), + false) } } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index e829de47ff31c..afd73c0eb0fd0 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -100,7 +100,8 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G // Clean out non-deterministic time schema info. case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _ => - val isOrdered = sharkQuery.executedPlan.collect { case s: Sort => s}.nonEmpty + // TODO: Really we only care about the final total ordering here... + val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(OrderedDistribution(_), _, _) => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer } From 2b9d80f2a1007510b5f94724db7331cfb2a6c180 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 17 Jan 2014 18:43:09 -0500 Subject: [PATCH 10/34] initial commit of adding exchange operators to physical plans. --- .../scala/catalyst/execution/Exchange.scala | 129 ++++++++++++++++++ .../catalyst/execution/SharkInstance.scala | 11 +- .../scala/catalyst/execution/SharkPlan.scala | 21 +++ .../scala/catalyst/execution/aggregates.scala | 13 +- .../catalyst/execution/basicOperators.scala | 34 +++-- .../catalyst/execution/dataProperties.scala | 106 ++++++++++++++ src/main/scala/catalyst/execution/joins.scala | 17 ++- .../execution/planningStrategies.scala | 8 +- src/main/scala/catalyst/plans/QueryPlan.scala | 30 +++- .../spark/rdd/SharkOrderedRDDFunctions.scala | 41 ++++++ .../spark/rdd/SharkPairRDDFunctions.scala | 81 +++++++++++ 11 files changed, 468 insertions(+), 23 deletions(-) create mode 100644 src/main/scala/catalyst/execution/Exchange.scala create mode 100644 src/main/scala/catalyst/execution/dataProperties.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala create mode 100644 src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala new file mode 100644 index 0000000000000..370f3a4ca2497 --- /dev/null +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -0,0 +1,129 @@ +package catalyst +package execution + +import catalyst.rules.Rule +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ + +import org.apache.spark.{RangePartitioner, HashPartitioner} +import org.apache.spark.rdd.ShuffledRDD + +case class Exchange( + dataProperty: Partitioned, + child: SharkPlan, + numPartitions: Int = 8) extends UnaryNode { + + def output = child.output + + def execute() = attachTree(this , "execute") { + dataProperty match { + case NotSpecified() => child.execute() + case g @ HashPartitioned(groupingExpressions) => { + val rdd = child.execute().map { row => + (buildRow(groupingExpressions.toSeq.map(Evaluate(_, Vector(row)))), row) + } + val part = new HashPartitioner(numPartitions) + val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) + + shuffled.map(_._2) + } + case s @ RangePartitioned(sortingExpressions) => { + val directions = sortingExpressions.map(_.direction).toIndexedSeq + val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + + class SortKey(val keyValues: IndexedSeq[Any]) + extends Ordered[SortKey] + with Serializable { + def compare(other: SortKey): Int = { + var i = 0 + while (i < keyValues.size) { + val left = keyValues(i) + val right = other.keyValues(i) + val curDirection = directions(i) + val curDataType = dataTypes(i) + + logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") + // TODO: Use numeric here too? + val comparison = + if (left == null && right == null) { + 0 + } else if (left == null) { + if (curDirection == Ascending) -1 else 1 + } else if (right == null) { + if (curDirection == Ascending) 1 else -1 + } else if (curDataType == IntegerType) { + if (curDirection == Ascending) { + left.asInstanceOf[Int] compare right.asInstanceOf[Int] + } else { + right.asInstanceOf[Int] compare left.asInstanceOf[Int] + } + } else if (curDataType == DoubleType) { + if (curDirection == Ascending) { + left.asInstanceOf[Double] compare right.asInstanceOf[Double] + } else { + right.asInstanceOf[Double] compare left.asInstanceOf[Double] + } + } else if (curDataType == StringType) { + if (curDirection == Ascending) { + left.asInstanceOf[String] compare right.asInstanceOf[String] + } else { + right.asInstanceOf[String] compare left.asInstanceOf[String] + } + } else { + sys.error(s"Comparison not yet implemented for: $curDataType") + } + + if (comparison != 0) return comparison + i += 1 + } + return 0 + } + } + + val rdd = child.execute().map { row => + val input = Vector(row) + val sortKey = new SortKey( + sortingExpressions.map(s => Evaluate(s.child, input)).toIndexedSeq) + + (sortKey, row) + } + val part = new RangePartitioner(numPartitions, rdd, ascending = true) + val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) + + shuffled.map(_._2) + } + } + } +} + +object AddExchange extends Rule[SharkPlan] { + def apply(plan: SharkPlan): SharkPlan = { + // TODO: determine the number of partitions. + // TODO: We need to consider the number of partitions to determine if we + // will add an Exchange operator. If a dataset only has a single partition, + // even if needExchange returns true, we do not need to shuffle the data again. + val numPartitions = 8 + plan.transformUp { + case operator: SharkPlan => { + val newChildren: Seq[SharkPlan] = operator.children.view.zipWithIndex.map { + case (child,index) => { + if (child.outputPartitioningScheme.needExchange( + operator.requiredPartitioningSchemes(index))) { + val exchange = new Exchange( + operator.requiredPartitioningSchemes(index), + child, + numPartitions) + + exchange + } else { + child + } + } + } + + operator.withNewChildren(newChildren) + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkInstance.scala b/src/main/scala/catalyst/execution/SharkInstance.scala index d5929d54d2418..122f2031c8def 100644 --- a/src/main/scala/catalyst/execution/SharkInstance.scala +++ b/src/main/scala/catalyst/execution/SharkInstance.scala @@ -82,7 +82,16 @@ abstract class SharkInstance extends Logging { } object PrepareForExecution extends RuleExecutor[SharkPlan] { - val batches = Batch("Prepare Expressions", Once, expressions.BindReferences) :: Nil + // TODO: [[catalyst.execution.AddExchange]] should be in a phase before + // PrepareForExecution. We may also need a phase of physical optimizations. + // In this phase, we can alter the data property in an Exchange operator to + // reduce the number of shuffling phases. Or, this physical optimization phase can be + // a part of QueryPlanner when a planner can accept a physical operator (a SharkPlan). + val batches = + Batch("Add exchange", Once, + AddExchange) :: + Batch("Prepare Expressions", Once, + expressions.BindReferences) :: Nil } class SharkSqlQuery(sql: String) extends SharkQuery { diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 6116d8397b090..83d50e4990e5c 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -4,10 +4,14 @@ package execution import org.apache.spark.rdd.RDD import catalyst.plans.QueryPlan +import scala.reflect.ClassTag abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => + def requiredPartitioningSchemes: Seq[Partitioned] + def outputPartitioningScheme: Partitioned + /** * Runs this query returning the result as an RDD. */ @@ -23,12 +27,29 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { self: Product => + + def requiredPartitioningScheme: Partitioned = NotSpecified() + def requiredPartitioningSchemes: Seq[Partitioned] = Seq(requiredPartitioningScheme) + // TODO: We should get the output data properties of a leaf node from metadata. + def outputPartitioningScheme: Partitioned = NotSpecified() } trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => + + def requiredPartitioningScheme: Partitioned = NotSpecified() + def requiredPartitioningSchemes: Seq[Partitioned] = Seq(requiredPartitioningScheme) + + def outputPartitioningScheme: Partitioned = child.outputPartitioningScheme } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { self: Product => + + def leftRequiredPartitioningScheme: Partitioned = NotSpecified() + def rightRequiredPartitioningScheme: Partitioned = NotSpecified() + def requiredPartitioningSchemes: Seq[Partitioned] = + Seq(leftRequiredPartitioningScheme, rightRequiredPartitioningScheme) + + def outputPartitioningScheme: Partitioned = NotSpecified() } diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 81ccaafd10d24..959770045a10e 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,16 +3,23 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ import org.apache.spark.SparkContext._ +import SharkPairRDDFunctions._ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SharkPlan) + (override val outputPartitioningScheme: Partitioned = + HashPartitioned(groupingExpressions)) extends UnaryNode { + override val requiredPartitioningScheme: Partitioned = HashPartitioned(groupingExpressions) + override def otherCopyArgs = outputPartitioningScheme :: Nil + case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -86,9 +93,12 @@ case class Aggregate( def output = aggregateExpressions.map(_.toAttribute) def execute() = attachTree(this, "execute") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the groupingExpressions again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. val grouped = child.execute().map { row => (buildRow(groupingExpressions.map(Evaluate(_, Vector(row)))), row) - }.groupByKey() + }.groupByKeyLocally() grouped.map { case (group, rows) => // Replace all aggregate expressions with spark functions that will compute the result. @@ -142,6 +152,7 @@ case class SparkAggregate(aggregateExprs: Seq[NamedExpression], child: SharkPlan (@transient sc: SharkContext) extends UnaryNode { def output = aggregateExprs.map(_.toAttribute) override def otherCopyArgs = Seq(sc) + override val requiredPartitioningScheme: Partitioned = HashPartitioned(Nil) case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 75f5808618df7..8f8b4b39e3e16 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -1,12 +1,12 @@ package catalyst package execution -import errors._ -import expressions._ -import types._ - -import org.apache.spark.SparkContext._ +import catalyst.errors._ +import catalyst.expressions._ +import catalyst.types._ +/* Implicits */ +import org.apache.spark.rdd.SharkOrderedRDDFunctions._ case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -45,12 +45,21 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { +case class Sort( + sortExprs: Seq[SortOrder], + child: SharkPlan) + (override val outputPartitioningScheme: Partitioned = + RangePartitioned(sortExprs)) extends UnaryNode { val numPartitions = 1 // TODO: Set with input cardinality + override val requiredPartitioningScheme: Partitioned = RangePartitioned(sortExprs) + override def otherCopyArgs = outputPartitioningScheme :: Nil + private final val directions = sortExprs.map(_.direction).toIndexedSeq private final val dataTypes = sortExprs.map(_.dataType).toIndexedSeq + // TODO: This SortKey and the one in [[catalyst.execute.Exchange]] are the same. + // We should just define it only once. private class SortKey(val keyValues: IndexedSeq[Any]) extends Ordered[SortKey] with Serializable { def compare(other: SortKey): Int = { var i = 0 @@ -106,12 +115,16 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Don't include redundant expressions in both sortKey and row. def execute() = attachTree(this, "sort") { + // TODO: If the child of it is an [[catalyst.execution.Exchange]], + // do not evaluate the sortKey again since we have evaluated it + // in the [[catalyst.execution.Exchange]]. child.execute().map { row => val input = Vector(row) - val sortKey = new SortKey(sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) + val sortKey = new SortKey( + sortExprs.map(s => Evaluate(s.child, input)).toIndexedSeq) (sortKey, row) - }.sortByKey(ascending = true, numPartitions).map(_._2) + }.sortByKeyLocally(ascending = true).map(_._2) } def output = child.output @@ -120,6 +133,9 @@ case class Sort(sortExprs: Seq[SortOrder], child: SharkPlan) extends UnaryNode { // TODO: Rename: SchemaRDD case class LocalRelation(output: Seq[Attribute], data: Seq[IndexedSeq[Any]]) (@transient sc: SharkContext) extends LeafNode { - def execute() = sc.makeRDD(data.map(buildRow), 1) + + // Since LocalRelation is used for unit tests, set the defaultParallelism to 2 + // to make sure we can cover bugs appearing in a distributed environment. + def execute() = sc.makeRDD(data.map(buildRow), 2) } diff --git a/src/main/scala/catalyst/execution/dataProperties.scala b/src/main/scala/catalyst/execution/dataProperties.scala new file mode 100644 index 0000000000000..2b67dd4242251 --- /dev/null +++ b/src/main/scala/catalyst/execution/dataProperties.scala @@ -0,0 +1,106 @@ +package catalyst +package execution + +import catalyst.expressions.{SortOrder, Expression} + +trait Partitioned { + def expressions: Seq[Expression] + + /** + * If we need an [[catalyst.execution.Exchange]] to re-partition data for + * the given [[catalyst.execution.Partitioned]] other. + * @param other The given [[catalyst.execution.Partitioned]]. + *@return + */ + // TODO: We should also consider functional dependencies between expressions of + // two data properties. For example, if we have a GroupProperty(a) and a + // GroupProperty(a+1), we will not need an exchange to re-partition the data. + def needExchange(other: Partitioned): Boolean +} + +/** + * An implementation of [[catalyst.execution.Partitioned]] represents that + * the data property of a dataset is not specified. + * If it is used as a required data property for a physical operator + * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), + * it means that this operator does not require its input datasets to be + * organized in a certain way. + */ +case class NotSpecified() extends Partitioned { + def expressions = Nil + + def needExchange(other: Partitioned): Boolean = { + other match { + case NotSpecified() => false + case HashPartitioned(groupingExpressions) => true + case RangePartitioned(_) => true + } + } +} + +/** + * An implementation of [[catalyst.execution.Partitioned]] represents that + * a dataset is grouped by groupingExpressions. + * @param groupingExpressions The expressions used to specify the way how rows should be grouped. + * If it is a Nil, the entire dataset is considered as a single group. + * In this case, a single reducer will be used. + */ +case class HashPartitioned(groupingExpressions: Seq[Expression]) extends Partitioned { + override val expressions = groupingExpressions + + def needExchange(other: Partitioned): Boolean = { + other match { + case NotSpecified() => false + // We do not need an Exchange operator if another GroupProperty only + // needs to group rows within a partition. + case g @ HashPartitioned(otherExpressions) => { + if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true + } + // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, + // we need to use an Exchange operator to sort data with a + // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is + // a Nil, we will have a single partition. So, we do not need an Exchange operator + // to sort this single partition. + case RangePartitioned(otherExpressions) => if (expressions == Nil) false else true + } + } +} + +/** + * An implementation of [[catalyst.execution.Partitioned]] represents that + * a dataset is sorted by sortingExpressions. A SortProperty also implies that + * the dataset is grouped by sortingExpressions. + * @param sortingExpressions The expressions used to specify the way how rows should be sorted. + * sortingExpressions should not be empty. + */ +case class RangePartitioned(sortingExpressions: Seq[SortOrder]) extends Partitioned { + + { + if (sortingExpressions == Nil) { + throw new IllegalArgumentException("Sorting expressions of a SortProperty " + + "are not specified.") + } + } + + def expressions = sortingExpressions + + def needExchange(other: Partitioned): Boolean = { + other match { + case NotSpecified() => false + // A SortProperty implies a GroupProperty. We do not need an Exchange operator + // if the GroupProperty only needs to group rows within a partition. + case g @ HashPartitioned(otherExpressions) => { + if (expressions.map(expression => expression.child). + toSet.subsetOf(otherExpressions.toSet)) false else true + } + // We do not need an Exchange operator if another SortProperty only needs to + // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) + // or we do not need to sort again (cases satisfying + // expressions.startsWith(otherExpressions)). + case s @ RangePartitioned(otherExpressions) => { + if (otherExpressions.startsWith(expressions) || + expressions.startsWith(otherExpressions)) false else true + } + } + } +} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index fc5b745781642..1f76ff0f3245b 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -9,33 +9,38 @@ import errors._ import expressions._ import plans._ -/* Implicits */ -import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.SharkPairRDDFunctions._ case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, right: SharkPlan) + (override val outputPartitioningScheme: Partitioned = + HashPartitioned(leftKeys)) extends BinaryNode { def output = left.output ++ right.output + override val leftRequiredPartitioningScheme: Partitioned = HashPartitioned(leftKeys) + override val rightRequiredPartitioningScheme: Partitioned = HashPartitioned(rightKeys) + override def otherCopyArgs = outputPartitioningScheme :: Nil + def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute .map { row => + val leftWithKeys = left.execute.map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) logger.debug(s"Generated left join keys ($leftKeys) => ($joinKeys) given row $row") (joinKeys, row) } - val rightWithKeys = right.execute().map { row => + val rightWithKeys = right.execute.map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) logger.debug(s"Generated right join keys ($rightKeys) => ($joinKeys) given row $row") (joinKeys, row) } // Do the join. - val joined = filterNulls(leftWithKeys).join(filterNulls(rightWithKeys)) + val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys)) // Drop join keys and merge input tuples. joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) } } @@ -63,7 +68,7 @@ case class BroadcastNestedLoopJoin( (@transient sc: SharkContext) extends BinaryNode { - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sc :: Nil def output = left.output ++ right.output diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 2d00f3ccb7972..6b859c5e0f48d 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -88,7 +88,7 @@ trait PlanningStrategies { val rightKeys = joinKeys.map(_._2) val joinOp = execution.SparkEquiInnerJoin( - leftKeys, rightKeys, planLater(left), planLater(right)) + leftKeys, rightKeys, planLater(left), planLater(right))() // Make sure other conditions are met if present. if (otherPredicates.nonEmpty) { @@ -135,16 +135,16 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil // TODO: It is correct, but overkill to do a global sorting here. case logical.SortPartitions(sortExprs, child) => - execution.Sort(sortExprs, planLater(child)) :: Nil + execution.Sort(sortExprs, planLater(child))() :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(group, agg, planLater(child)) :: Nil + execution.Aggregate(group, agg, planLater(child))() :: Nil case logical.LocalRelation(output, data) => execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => diff --git a/src/main/scala/catalyst/plans/QueryPlan.scala b/src/main/scala/catalyst/plans/QueryPlan.scala index 3546416402f11..cfc84aa78c6a2 100644 --- a/src/main/scala/catalyst/plans/QueryPlan.scala +++ b/src/main/scala/catalyst/plans/QueryPlan.scala @@ -1,8 +1,10 @@ package catalyst package plans -import expressions.{Attribute, Expression} -import trees._ +import catalyst.expressions.{SortOrder, Attribute, Expression} +import catalyst.trees._ +import catalyst.execution.HashPartitioned +import catalyst.execution.RangePartitioned abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -48,6 +50,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case other => other } + case g @ HashPartitioned(groupingExpressions) => HashPartitioned( + groupingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}) + case g @ RangePartitioned(sortingExpressions) => RangePartitioned( + sortingExpressions.map { + case e: Expression => transformExpressionDown(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -79,6 +89,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case other => other } + case g @ HashPartitioned(groupingExpressions) => HashPartitioned( + groupingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}) + case g @ RangePartitioned(sortingExpressions) => RangePartitioned( + sortingExpressions.map { + case e: Expression => transformExpressionUp(e) + case other => other}.asInstanceOf[Seq[SortOrder]]) case other: AnyRef => other }.toArray @@ -102,6 +120,14 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => e :: Nil case other => Nil } + case g @ HashPartitioned(groupingExpressions) => groupingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } + case g @ RangePartitioned(sortingExpressions) => sortingExpressions.flatMap { + case e: Expression => e :: Nil + case other => Nil + } case other => Nil }.toSeq } diff --git a/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala new file mode 100644 index 0000000000000..bd33b6bcd00eb --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkOrderedRDDFunctions.scala @@ -0,0 +1,41 @@ +package org.apache.spark.rdd + +import scala.reflect.ClassTag +import org.apache.spark.Logging + +import scala.language.implicitConversions + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs where the key is + * sortable through an implicit conversion. + * Import `org.apache.spark.SharkOrderedRDDFunctions._` at the top of your program to + * use these functions. They will work with any key type that has a `scala.math.Ordered` + * implementation. + */ +class SharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, + V: ClassTag, + P <: Product2[K, V] : ClassTag]( + self: RDD[P]) + extends Logging with Serializable { + + /** + * Sort data within a partition. This function will not introduce a + * shuffling operation. + */ + def sortByKeyLocally(ascending: Boolean = true): RDD[P] = { + self.mapPartitions(iter => { + val buf = iter.toArray + if (ascending) { + buf.sortWith((x, y) => x._1 < y._1).iterator + } else { + buf.sortWith((x, y) => x._1 > y._1).iterator + } + }, preservesPartitioning = true) + } +} + +object SharkOrderedRDDFunctions { + implicit def rddToSharkOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( + rdd: RDD[(K, V)]) = + new SharkOrderedRDDFunctions[K, V, (K, V)](rdd) +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala new file mode 100644 index 0000000000000..ad9e93165e790 --- /dev/null +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -0,0 +1,81 @@ +package org.apache.spark.rdd + +import scala.reflect._ +import org.apache.spark._ +import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Aggregator +import org.apache.spark.SparkContext._ + +import scala.language.implicitConversions +import org.apache.spark.util.AppendOnlyMap + +/** + * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. + * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to use these functions. + */ +class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) + extends Logging + with Serializable { + + /** + * Cogroup corresponding partitions of `this` and `other`. These two RDDs should have + * the same number of partitions. Partitions of these two RDDs are cogrouped + * according to the indexes of partitions. If we have two RDDs and + * each of them has n partitions, we will cogroup the partition i from `this` + * with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + val cg = self.zipPartitions(other)((iter1:Iterator[(K, V)], iter2:Iterator[(K, W)]) => { + val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + + val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + if (hadVal) oldVal else Array.fill(2)(new ArrayBuffer[Any]) + } + + val getSeq = (k: K) => { + map.changeValue(k, update) + } + + iter1.foreach { kv => getSeq(kv._1)(0) += kv._2 } + iter2.foreach { kv => getSeq(kv._1)(1) += kv._2 } + + map.iterator + }).mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])} + + cg + } + + /** + * Group the values for each key within a partition of the RDD into a single sequence. + * This function will not introduce a shuffling operation. + */ + def groupByKeyLocally(): RDD[(K, Seq[V])] = { + def createCombiner(v: V) = ArrayBuffer(v) + def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v + val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner _, mergeValue _, null) + val bufs = self.mapPartitionsWithContext((context, iter) => { + new InterruptibleIterator(context, aggregator.combineValuesByKey(iter)) + }, preservesPartitioning = true) + bufs.asInstanceOf[RDD[(K, Seq[V])]] + } + + /** + * Join corresponding partitions of `this` and `other`. + * If we have two RDDs and each of them has n partitions, + * we will join the partition i from `this` with the partition i from `other`. + * This function will not introduce a shuffling operation. + */ + def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + cogroupLocally(other).flatMapValues { + case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } +} + +object SharkPairRDDFunctions { + implicit def rddToSharkPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + new SharkPairRDDFunctions(rdd) +} + + From 73c70dea02e0aa820f9e70b377ffa7f811144fcc Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 17 Jan 2014 18:44:18 -0500 Subject: [PATCH 11/34] add a first set of unit tests for data properties. --- .../execution/DataPropertyTests.scala | 154 ++++++++++++++++++ 1 file changed, 154 insertions(+) create mode 100644 src/test/scala/catalyst/execution/DataPropertyTests.scala diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala new file mode 100644 index 0000000000000..67cd333393136 --- /dev/null +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -0,0 +1,154 @@ +package catalyst.execution + +import catalyst.dsl._ + +import org.scalatest.FunSuite + +class DataPropertyTests extends FunSuite { + + protected def checkNeedExchange( + outputDataProperty: Partitioned, + inputDataProperty: Partitioned, + expected: Boolean) { + assert( + outputDataProperty.needExchange(inputDataProperty) === expected, + s""" + |== Output data property == + |$outputDataProperty + |== Input data property == + |$inputDataProperty + |== Expected result of needExchange == + |$expected + """.stripMargin) + } + + test("needExchange test: GroupProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + NotSpecified(), + false) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + HashPartitioned(Seq('b, 'c)), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + HashPartitioned(Nil), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + HashPartitioned(Nil), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + // Cases which need an exchange between two data properties. + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Seq('b, 'c)), + true) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Seq('d, 'e)), + true) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + HashPartitioned(Nil), + true) + + checkNeedExchange( + HashPartitioned(Seq('a, 'b, 'c)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + checkNeedExchange( + HashPartitioned(Seq('b, 'c)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + // TODO: We should check functional dependencies + /* + checkNeedExchange( + GroupProperty(Seq('b)), + GroupProperty(Seq('b + 1)), + false) + */ + } + + test("needExchange test: SortProperty is the output DataProperty") { + // Cases which do not need an exchange between two data properties. + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + NotSpecified(), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.asc)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('a, 'b, 'c)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('c, 'b, 'a)), + false) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('b, 'c, 'a, 'd)), + false) + + // Cases which need an exchange between two data properties. + // TODO: We can have an optimization to first sort the dataset + // by a.asc and then sort b, and c in a partition. This optimization + // should tradeoff the benefit of a less number of Exchange operators + // and the parallelism. + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('a.asc, 'b.desc, 'c.asc)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioned(Seq('b.asc, 'a.asc)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('a, 'b)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Seq('c, 'd)), + true) + + checkNeedExchange( + RangePartitioned(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioned(Nil), + true) + } +} \ No newline at end of file From 583a337838824ad4621471975136e35afe9969ad Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 18 Jan 2014 18:20:30 -0800 Subject: [PATCH 12/34] break apart distribution and partitioning. --- .../scala/catalyst/execution/Exchange.scala | 52 +++++-- .../scala/catalyst/execution/SharkPlan.scala | 6 +- .../scala/catalyst/execution/aggregates.scala | 3 +- .../catalyst/execution/dataProperties.scala | 106 ------------- .../catalyst/execution/distribution.scala | 88 ----------- src/main/scala/catalyst/execution/joins.scala | 3 +- .../execution/planningStrategies.scala | 3 +- .../plans/physical/partitioning.scala | 143 ++++++++++++++++++ .../spark/rdd/CoGroupedLocallyRDD.scala | 2 +- .../execution/DataPropertyTests.scala | 4 + .../execution/HiveComparisonTest.scala | 4 +- 11 files changed, 197 insertions(+), 217 deletions(-) create mode 100644 src/main/scala/catalyst/plans/physical/partitioning.scala diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 14c4019cdc30e..764c106912786 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -4,12 +4,13 @@ package execution import catalyst.rules.Rule import catalyst.errors._ import catalyst.expressions._ +import catalyst.plans.physical._ import catalyst.types._ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD -case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: SharkPlan) +case class Exchange(newPartitioning: Partitioning, child: SharkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning @@ -17,17 +18,17 @@ case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: Sh def execute() = attachTree(this , "execute") { newPartitioning match { - case ClusteredDistribution(expressions) => + case HashPartitioning(expressions, width) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) } - val part = new HashPartitioner(numPartitions) + val part = new HashPartitioner(width) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - case OrderedDistribution(sortingExpressions) => + case RangePartitioning(sortingExpressions, width) => val directions = sortingExpressions.map(_.direction).toIndexedSeq val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq @@ -94,30 +95,49 @@ case class Exchange(newPartitioning: Distribution, numPartitions: Int, child: Sh (sortKey, row) } - val part = new RangePartitioner(numPartitions, rdd, ascending = true) + val part = new RangePartitioner(width, rdd, ascending = true) val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) shuffled.map(_._2) - case UnknownDistribution => - logger.warn("Worthless repartitioning requested.") - child.execute() + case _ => sys.error("Not implemented") } } } object AddExchange extends Rule[SharkPlan] { // TODO: determine the number of partitions. - // TODO: We need to consider the number of partitions to determine if we - // will add an Exchange operator. If a dataset only has a single partition, - // even if needExchange returns true, we do not need to shuffle the data again. val numPartitions = 8 def apply(plan: SharkPlan): SharkPlan = plan.transformUp { case operator: SharkPlan => - operator.withNewChildren(operator.requiredChildPartitioning.zip(operator.children).map { - case (required, child) if !child.outputPartitioning.satisfies(required) => - Exchange(required, numPartitions, child) - case (_, child) => child - }) + def meetsRequirements = + !operator.requiredChildDistribution.zip(operator.children).map { + case (required, child) => !child.outputPartitioning.satisfies(required) + }.exists(_ == false) + + // TODO ASUUMES TRANSITIVITY? + def compatible = + !operator.children + .map(_.outputPartitioning) + .sliding(2) + .map { + case Seq(a) => true + case Seq(a,b) => a compatibleWith b + }.exists(_ == false) + + + if (false && meetsRequirements && compatible) { + operator + } else { + val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { + case (ClusteredDistribution(clustering), child) => + Exchange(HashPartitioning(clustering, 8), child) + case (OrderedDistribution(ordering), child) => + Exchange(RangePartitioning(ordering, 8), child) + case (UnknownDistribution, child) => child + case (dist, _) => sys.error(s"Don't know how to ensure $dist") + } + operator.withNewChildren(repartitionedChildren) + } } } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index fc46b1494d8a5..ba34a3c178e77 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -4,14 +4,16 @@ package execution import org.apache.spark.rdd.RDD import plans.QueryPlan +import plans.physical._ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => + // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ - def outputPartitioning: Distribution = UnknownDistribution + def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG! /** Specifies any partition requirements on the input data for this operator. */ - def requiredChildPartitioning: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) + def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) /** * Runs this query returning the result as an RDD. diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 89fa0912bb147..99a67d5c0ff64 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,6 +3,7 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import catalyst.plans.physical._ import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ @@ -16,7 +17,7 @@ case class Aggregate( extends UnaryNode { val requiredPartitioning = ClusteredDistribution(groupingExpressions) - override def requiredChildPartitioning = Seq(requiredPartitioning, requiredPartitioning) + override def requiredChildDistribution = Seq(requiredPartitioning, requiredPartitioning) def output = aggregateExpressions.map(_.toAttribute) diff --git a/src/main/scala/catalyst/execution/dataProperties.scala b/src/main/scala/catalyst/execution/dataProperties.scala index 2b67dd4242251..e69de29bb2d1d 100644 --- a/src/main/scala/catalyst/execution/dataProperties.scala +++ b/src/main/scala/catalyst/execution/dataProperties.scala @@ -1,106 +0,0 @@ -package catalyst -package execution - -import catalyst.expressions.{SortOrder, Expression} - -trait Partitioned { - def expressions: Seq[Expression] - - /** - * If we need an [[catalyst.execution.Exchange]] to re-partition data for - * the given [[catalyst.execution.Partitioned]] other. - * @param other The given [[catalyst.execution.Partitioned]]. - *@return - */ - // TODO: We should also consider functional dependencies between expressions of - // two data properties. For example, if we have a GroupProperty(a) and a - // GroupProperty(a+1), we will not need an exchange to re-partition the data. - def needExchange(other: Partitioned): Boolean -} - -/** - * An implementation of [[catalyst.execution.Partitioned]] represents that - * the data property of a dataset is not specified. - * If it is used as a required data property for a physical operator - * (a [[catalyst.execution.SharkPlan]]) (e.g. [[catalyst.execution.Project]]), - * it means that this operator does not require its input datasets to be - * organized in a certain way. - */ -case class NotSpecified() extends Partitioned { - def expressions = Nil - - def needExchange(other: Partitioned): Boolean = { - other match { - case NotSpecified() => false - case HashPartitioned(groupingExpressions) => true - case RangePartitioned(_) => true - } - } -} - -/** - * An implementation of [[catalyst.execution.Partitioned]] represents that - * a dataset is grouped by groupingExpressions. - * @param groupingExpressions The expressions used to specify the way how rows should be grouped. - * If it is a Nil, the entire dataset is considered as a single group. - * In this case, a single reducer will be used. - */ -case class HashPartitioned(groupingExpressions: Seq[Expression]) extends Partitioned { - override val expressions = groupingExpressions - - def needExchange(other: Partitioned): Boolean = { - other match { - case NotSpecified() => false - // We do not need an Exchange operator if another GroupProperty only - // needs to group rows within a partition. - case g @ HashPartitioned(otherExpressions) => { - if (expressions.toSet.subsetOf(otherExpressions.toSet)) false else true - } - // Because we use [[org.apache.spark.HashPartitioner]] for GroupProperty, - // we need to use an Exchange operator to sort data with a - // [[org.apache.spark.RangePartitioner]]. But, if the groupingExpressions is - // a Nil, we will have a single partition. So, we do not need an Exchange operator - // to sort this single partition. - case RangePartitioned(otherExpressions) => if (expressions == Nil) false else true - } - } -} - -/** - * An implementation of [[catalyst.execution.Partitioned]] represents that - * a dataset is sorted by sortingExpressions. A SortProperty also implies that - * the dataset is grouped by sortingExpressions. - * @param sortingExpressions The expressions used to specify the way how rows should be sorted. - * sortingExpressions should not be empty. - */ -case class RangePartitioned(sortingExpressions: Seq[SortOrder]) extends Partitioned { - - { - if (sortingExpressions == Nil) { - throw new IllegalArgumentException("Sorting expressions of a SortProperty " + - "are not specified.") - } - } - - def expressions = sortingExpressions - - def needExchange(other: Partitioned): Boolean = { - other match { - case NotSpecified() => false - // A SortProperty implies a GroupProperty. We do not need an Exchange operator - // if the GroupProperty only needs to group rows within a partition. - case g @ HashPartitioned(otherExpressions) => { - if (expressions.map(expression => expression.child). - toSet.subsetOf(otherExpressions.toSet)) false else true - } - // We do not need an Exchange operator if another SortProperty only needs to - // sort rows within a partition (cases satisfying otherExpressions.startsWith(expressions)) - // or we do not need to sort again (cases satisfying - // expressions.startsWith(otherExpressions)). - case s @ RangePartitioned(otherExpressions) => { - if (otherExpressions.startsWith(expressions) || - expressions.startsWith(otherExpressions)) false else true - } - } - } -} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/distribution.scala b/src/main/scala/catalyst/execution/distribution.scala index 4ffd5a3f61cf7..e69de29bb2d1d 100644 --- a/src/main/scala/catalyst/execution/distribution.scala +++ b/src/main/scala/catalyst/execution/distribution.scala @@ -1,88 +0,0 @@ -package catalyst -package execution - -import expressions.{Expression, SortOrder} -import types._ - -/** - * Specifies how tuples that share common expressions will be distributed. Distribution can - * be used to refer to two distinct physical properties: - * - Inter-node partitioning of data: In this case the distribution describes how tuples are - * partitioned across physical machines in a cluster. Knowing this property allows some - * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. - * - Intra-partition ordering of data: In this case the distribution describes guarantees made - * about how tuples are distributed within a single partition. - * - * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping - * actually does matter, and thus our subset checking is probably not sufficient - * to ensure correct colocation for joins.? - * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort - * aggregation can be done. Maybe we need two checks? Separate ordering from - * partitioning? - */ -abstract sealed class Distribution extends Expression { - self: Product => - - /** - * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all - * guarantees mandated by the `required` distribution. - */ - def satisfies(required: Distribution): Boolean -} - -/** - * Represents a distribution where no promises are made about co-location of data. - */ -case object UnknownDistribution extends Distribution with trees.LeafNode[Expression] { - def references = Set.empty - def nullable = false - def dataType = NullType - - def satisfies(required: Distribution): Boolean = required == UnknownDistribution -} - -/** - * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will - * be co-located. Based on the context, this can mean such tuples are either co-located in the same - * partition or they will be contiguous within a single partition. - */ -case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { - def children = clustering.toSeq - def references = clustering.flatMap(_.references).toSet - def nullable = false - def dataType = NullType - - def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true - // No clustering expressions means only one partition. - case _ if clustering.isEmpty => true - case ClusteredDistribution(requiredClustering) => - clustering.toSet.subsetOf(requiredClustering.toSet) - case _ => false - } -} - -/** - * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This - * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that - * tuples that share the same value for the ordering expressions are contiguous and will never be - * split across partitions. - */ -case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { - def children = ordering - def references = ordering.flatMap(_.references).toSet - def nullable = false - def dataType = NullType - - def clustering = ordering.map(_.child).toSet - - def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true - case OrderedDistribution(requiredOrdering) => - val minSize = Seq(requiredOrdering.size, ordering.size).min - requiredOrdering.take(minSize) == ordering.take(minSize) - case ClusteredDistribution(requiredClustering) => - clustering.subsetOf(requiredClustering.toSet) - case _ => false - } -} \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 6f811f20a737d..380f64dee8753 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -8,6 +8,7 @@ import org.apache.spark.rdd.RDD import errors._ import expressions._ import plans._ +import plans.physical._ import org.apache.spark.rdd.SharkPairRDDFunctions._ @@ -18,7 +19,7 @@ case class SparkEquiInnerJoin( right: SharkPlan) extends BinaryNode { - override def requiredChildPartitioning = + override def requiredChildDistribution= ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil def output = left.output ++ right.output diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index bcce25697ac84..5cdb7c3016275 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -5,6 +5,7 @@ import expressions._ import planning._ import plans._ import plans.logical.LogicalPlan +import plans.physical._ trait PlanningStrategies { self: QueryPlanner[SharkPlan] => @@ -137,7 +138,7 @@ trait PlanningStrategies { case logical.Sort(sortExprs, child) => // First repartition then sort locally. execution.SortPartitions(sortExprs, - execution.Exchange(OrderedDistribution(sortExprs), 8, planLater(child))) :: Nil + execution.Exchange(RangePartitioning(sortExprs, 8), planLater(child))) :: Nil case logical.SortPartitions(sortExprs, child) => execution.SortPartitions(sortExprs, planLater(child)) :: Nil case logical.Project(projectList, child) => diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala new file mode 100644 index 0000000000000..2c5e15d6a7fe9 --- /dev/null +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -0,0 +1,143 @@ +package catalyst +package plans +package physical + +import expressions._ +import types._ + +/** + * Specifies how tuples that share common expressions will be distributed. Distribution can + * be used to refer to two distinct physical properties: + * - Inter-node partitioning of data: In this case the distribution describes how tuples are + * partitioned across physical machines in a cluster. Knowing this property allows some + * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. + * - Intra-partition ordering of data: In this case the distribution describes guarantees made + * about how tuples are distributed within a single partition. + * + * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping + * actually does matter, and thus our subset checking is probably not sufficient + * to ensure correct colocation for joins.? + * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort + * aggregation can be done. Maybe we need two checks? Separate ordering from + * partitioning? + */ +sealed trait Distribution { +} + +/** + * Represents a distribution where no promises are made about co-location of data. + */ +case object UnknownDistribution extends Distribution + +/** + * Represents a distribution where a single operation can observe all tuples in the dataset. + */ +case object AllTuples extends Distribution + +/** + * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will + * be co-located. Based on the context, this can mean such tuples are either co-located in the same + * partition or they will be contiguous within a single partition. + */ +case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution + +/** + * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This + * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that + * tuples that share the same value for the ordering expressions are contiguous and will never be + * split across partitions. + */ +case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { + def clustering = ordering.map(_.child).toSet +} + +sealed abstract trait Partitioning { + self: Product => + + val width: Int + + /** + * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all + * guarantees mandated by the `required` distribution. + */ + def satisfies(required: Distribution): Boolean + + def compatibleWith(other: Partitioning): Boolean +} + +case class UnknownPartitioning(width: Int) extends Partitioning { + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case _ => false + } + def compatibleWith(other: Partitioning): Boolean = other match { + case UnknownPartitioning(_) => true + case _ => false + } +} + +case object Unpartitioned extends Partitioning { + val width = 1 + + override def satisfies(required: Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { + case Unpartitioned => true + case _ => false + } +} + +case object Broadcast extends Partitioning { + val width = 1 + + override def satisfies(required:Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { + case Unpartitioned => true + case _ => false + } +} + +case class HashPartitioning(expressions: Seq[Expression], width: Int) extends Expression with Partitioning { + def children = expressions.toSeq + def references = expressions.flatMap(_.references).toSet + def nullable = false + def dataType = IntegerType + + lazy val clusteringSet = expressions.toSet + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case ClusteredDistribution(requiredClustering) => + clusteringSet.subsetOf(requiredClustering.toSet) + case _ => false + } + override def compatibleWith(other: Partitioning) = other match { + case Broadcast => true + case h: HashPartitioning if h == this => true + case _ => false + } +} + +case class RangePartitioning(ordering: Seq[SortOrder], width: Int) extends Expression with Partitioning { + def children = ordering.toSeq + def references = ordering.flatMap(_.references).toSet + def nullable = false + def dataType = IntegerType + + lazy val clusteringSet = ordering.map(_.child).toSet + + def satisfies(required: Distribution): Boolean = required match { + case UnknownDistribution => true + case OrderedDistribution(requiredOrdering) => + val minSize = Seq(requiredOrdering.size, ordering.size).min + requiredOrdering.take(minSize) == ordering.take(minSize) + case ClusteredDistribution(requiredClustering) => + clusteringSet.subsetOf(requiredClustering.toSet) + case _ => false + } + + def compatibleWith(other: Partitioning) = other match { + case Broadcast => true + case r: RangePartitioning if r == this => true + case _ => false + } +} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala index 7a7a6b61a05b0..a7b639aa61186 100644 --- a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala +++ b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala @@ -1,7 +1,7 @@ package org.apache.spark.rdd import org.apache.spark._ -import org.apache.spark.util.AppendOnlyMap +import org.apache.spark.util.collection.AppendOnlyMap import scala.collection.mutable.ArrayBuffer /** diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DataPropertyTests.scala index 933897cff3922..faf9c5504065e 100644 --- a/src/test/scala/catalyst/execution/DataPropertyTests.scala +++ b/src/test/scala/catalyst/execution/DataPropertyTests.scala @@ -5,6 +5,8 @@ import catalyst.dsl._ import org.scalatest.FunSuite class DistributionTests extends FunSuite { + /* + protected def checkSatisfied( inputDistribution: Distribution, requiredDistribution: Distribution, @@ -150,4 +152,6 @@ class DistributionTests extends FunSuite { ClusteredDistribution(Nil), false) } + + */ } \ No newline at end of file diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index afd73c0eb0fd0..c98b2ebb50f6e 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -5,8 +5,10 @@ import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} import frontend.hive.{ExplainCommand, Command} +import plans.physical._ import util._ + /** * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. @@ -101,7 +103,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _ => // TODO: Really we only care about the final total ordering here... - val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(OrderedDistribution(_), _, _) => s}.nonEmpty + val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(r: RangePartitioning, _) => s}.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer } From 658866e86d42d7fa91d8b84821503d44a78bf8c1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 20 Jan 2014 14:34:56 -0800 Subject: [PATCH 13/34] Pull back in changes made by @yhuai eliminating CoGroupedLocallyRDD.scala --- .../spark/rdd/CoGroupedLocallyRDD.scala | 85 ------------------- .../spark/rdd/SharkPairRDDFunctions.scala | 28 ++++-- 2 files changed, 21 insertions(+), 92 deletions(-) delete mode 100644 src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala diff --git a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala b/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala deleted file mode 100644 index a7b639aa61186..0000000000000 --- a/src/main/scala/org/apache/spark/rdd/CoGroupedLocallyRDD.scala +++ /dev/null @@ -1,85 +0,0 @@ -package org.apache.spark.rdd - -import org.apache.spark._ -import org.apache.spark.util.collection.AppendOnlyMap -import scala.collection.mutable.ArrayBuffer - -/** - * A RDD that cogroups its already co-partitioned parents. This RDD works the same as - * a [[org.apache.spark.rdd.CoGroupedRDD]] except that its parents should have the - * same number of partitions. Like a [[org.apache.spark.rdd.CoGroupedRDD]], - * for each key k in parent RDDs, the resulting RDD contains a tuple with the list of - * values for that key. - * - * @param rdds parent RDDs. - */ -class CoGroupedLocallyRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]]) - extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { - - { - // Check if all parents have the same number of partitions. - // It is possible that a parent RDD does not preserve the partitioner, - // so we do not check if all of parent RDDs have the same partitioner. - if (!rdds.forall(rdd => rdd.partitions.size == rdds(0).partitions.size)) { - throw new IllegalArgumentException( - "All parent RDDs should have the same number of partitions.") - } - } - - // All dependencies of a CoGroupedLocallyRDD should be narrow dependencies. - override def getDependencies: Seq[Dependency[_]] = { - rdds.map { rdd: RDD[_ <: Product2[K, _]] => - logDebug("Adding one-to-one dependency with " + rdd) - new OneToOneDependency(rdd) - } - } - - override def getPartitions: Array[Partition] = { - val numPartitions = firstParent[(K, _)].partitions.size - val array = new Array[Partition](numPartitions) - for (i <- 0 until array.size) { - // Each CoGroupPartition will have a dependency per contributing RDD - array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => - new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) - }.toArray) - } - array - } - - // Take the first not None partitioner. It is possible that all parent partitioners - // are None. - override val partitioner = rdds.find(rdd => rdd.partitioner != None) match { - case Some(rdd) => rdd.partitioner - case None => None - } - - override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { - val split = s.asInstanceOf[CoGroupPartition] - val numRdds = split.deps.size - // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) - val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - - val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { - if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) - } - - val getSeq = (k: K) => { - map.changeValue(k, update) - } - - for ((dep, depNum) <- split.deps.zipWithIndex) dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { - // Read them from the parent - rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv => - getSeq(kv._1)(depNum) += kv._2 - } - } - case _ => { - // We should not reach here. It is a sanity check. - throw new RuntimeException("A dependency of this CoGroupedLocallyRDD is not " + - "a narrow dependency.") - } - } - new InterruptibleIterator(context, map.iterator) - } -} \ No newline at end of file diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala index d26a43ac5720b..67ecac8e90670 100644 --- a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -7,6 +7,7 @@ import org.apache.spark.Aggregator import org.apache.spark.SparkContext._ import scala.language.implicitConversions +import org.apache.spark.util.collection.AppendOnlyMap /** * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. @@ -25,11 +26,24 @@ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * This function will not introduce a shuffling operation. */ def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { - val cg = new CoGroupedLocallyRDD[K](Seq(self, other)) - val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag) - prfs.mapValues { case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) - } + val cg = self.zipPartitions(other)((iter1:Iterator[(K, V)], iter2:Iterator[(K, W)]) => { + val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + + val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + if (hadVal) oldVal else Array.fill(2)(new ArrayBuffer[Any]) + } + + val getSeq = (k: K) => { + map.changeValue(k, update) + } + + iter1.foreach { kv => getSeq(kv._1)(0) += kv._2 } + iter2.foreach { kv => getSeq(kv._1)(1) += kv._2 } + + map.iterator + }).mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])} + + cg } /** @@ -53,8 +67,8 @@ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * This function will not introduce a shuffling operation. */ def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { - cogroupLocally(other).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + cogroupLocally(other).flatMapValues { + case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) } } } From f2fdd7733bf4fcfb8cd37f31aeed2bf2cf521317 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 20 Jan 2014 15:12:10 -0800 Subject: [PATCH 14/34] fix required distribtion for aggregate. --- src/main/scala/catalyst/execution/aggregates.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 99a67d5c0ff64..a256c5f979dd8 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -16,8 +16,7 @@ case class Aggregate( child: SharkPlan) extends UnaryNode { - val requiredPartitioning = ClusteredDistribution(groupingExpressions) - override def requiredChildDistribution = Seq(requiredPartitioning, requiredPartitioning) + override def requiredChildDistribution = Seq(ClusteredDistribution(groupingExpressions)) def output = aggregateExpressions.map(_.toAttribute) From 9804eb5384a3ca8ece5496d71ed2cae2072183e2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 11:48:25 -0800 Subject: [PATCH 15/34] upgrade spark --- build.sbt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 2fe38ceb72ac8..28e9967946e3b 100644 --- a/build.sbt +++ b/build.sbt @@ -12,7 +12,10 @@ scalacOptions ++= Seq("-deprecation", "-feature", "-unchecked") resolvers += "Local Maven Repository" at "file://"+Path.userHome.absolutePath+"/.m2/repository" -libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating-SNAPSHOT" +// TODO: Remove when Spark 0.9.0 is released for real. +resolvers += "SparkStaging" at "https://repository.apache.org/content/repositories/orgapachespark-1005/" + +libraryDependencies += "org.apache.spark" %% "spark-core" % "0.9.0-incubating" libraryDependencies += "catalyst" % "hive-golden" % "4" % "test" from "http://repository-databricks.forge.cloudbees.com/snapshot/catalystGolden4.jar" From ece15e1412bb5f3fc7ebf1bbb24fdededc28c515 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:02:56 -0800 Subject: [PATCH 16/34] update unit tests --- .../plans/physical/partitioning.scala | 8 +-- ...rtyTests.scala => DistributionTests.scala} | 65 +++++++++---------- 2 files changed, 33 insertions(+), 40 deletions(-) rename src/test/scala/catalyst/execution/{DataPropertyTests.scala => DistributionTests.scala} (68%) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 2c5e15d6a7fe9..eae6aedec2686 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -13,13 +13,6 @@ import types._ * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. * - Intra-partition ordering of data: In this case the distribution describes guarantees made * about how tuples are distributed within a single partition. - * - * TOOD(marmbrus): Given the way that Spark does partitioning, I think the order of the grouping - * actually does matter, and thus our subset checking is probably not sufficient - * to ensure correct colocation for joins.? - * TODO(marmbrus): Similarly I'm not sure that satisfies is a sufficient check to see if an sort - * aggregation can be done. Maybe we need two checks? Separate ordering from - * partitioning? */ sealed trait Distribution { } @@ -54,6 +47,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { sealed abstract trait Partitioning { self: Product => + /** Returns the number of partitions that the data is split across */ val width: Int /** diff --git a/src/test/scala/catalyst/execution/DataPropertyTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala similarity index 68% rename from src/test/scala/catalyst/execution/DataPropertyTests.scala rename to src/test/scala/catalyst/execution/DistributionTests.scala index faf9c5504065e..61e074127d05b 100644 --- a/src/test/scala/catalyst/execution/DataPropertyTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -1,78 +1,79 @@ -package catalyst.execution - -import catalyst.dsl._ +package catalyst +package plans +package physical import org.scalatest.FunSuite +import catalyst.dsl._ + class DistributionTests extends FunSuite { - /* protected def checkSatisfied( - inputDistribution: Distribution, + inputPartitioning: Partitioning, requiredDistribution: Distribution, satisfied: Boolean) { - if(inputDistribution.satisfies(requiredDistribution) != satisfied) + if(inputPartitioning.satisfies(requiredDistribution) != satisfied) fail( s""" |== Input Distribution == - |$inputDistribution + |$inputPartitioning |== Required Distribution == |$requiredDistribution |== Does input distribution satisfy requirements? == - |Expected $satisfied got ${inputDistribution.satisfies(requiredDistribution)} + |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) } test("needExchange test: ClusteredDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), UnknownDistribution, true) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - ClusteredDistribution(Seq('b, 'c)), + HashPartitioning(Seq('b, 'c), 10), ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - ClusteredDistribution(Nil), + Unpartitioned, ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - ClusteredDistribution(Nil), + Unpartitioned, OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) // Cases which need an exchange between two data properties. checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), ClusteredDistribution(Seq('b, 'c)), false) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), ClusteredDistribution(Seq('d, 'e)), false) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), - ClusteredDistribution(Nil), + HashPartitioning(Seq('a, 'b, 'c), 10), + AllTuples, false) checkSatisfied( - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq('a, 'b, 'c), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) checkSatisfied( - ClusteredDistribution(Seq('b, 'c)), + HashPartitioning(Seq('b, 'c), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) @@ -88,37 +89,37 @@ class DistributionTests extends FunSuite { test("needExchange test: OrderedDistribution is the output DataProperty") { // Cases which do not need an exchange between two data properties. checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), UnknownDistribution, true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.asc)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('c, 'b, 'a)), true) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('b, 'c, 'a, 'd)), true) @@ -128,30 +129,28 @@ class DistributionTests extends FunSuite { // should tradeoff the benefit of a less number of Exchange operators // and the parallelism. checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), OrderedDistribution(Seq('b.asc, 'a.asc)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('a, 'b)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Seq('c, 'd)), false) checkSatisfied( - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), ClusteredDistribution(Nil), false) } - - */ } \ No newline at end of file From 4f345f232d5bb5dfee076bdb6da40827adb5ee55 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:54:14 -0800 Subject: [PATCH 17/34] Remove SortKey, use RowOrdering. --- .../scala/catalyst/execution/Exchange.scala | 76 ++----------------- 1 file changed, 7 insertions(+), 69 deletions(-) diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 764c106912786..1cda0a5d67a48 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -29,77 +29,15 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) shuffled.map(_._2) case RangePartitioning(sortingExpressions, width) => - val directions = sortingExpressions.map(_.direction).toIndexedSeq - val dataTypes = sortingExpressions.map(_.dataType).toIndexedSeq + // TODO: ShuffledRDD should take an Ordering. + import scala.math.Ordering.Implicits._ + implicit val ordering = new RowOrdering(sortingExpressions) - // TODO: MOVE THIS! - class SortKey(val keyValues: IndexedSeq[Any]) - extends Ordered[SortKey] - with Serializable { - def compare(other: SortKey): Int = { - var i = 0 - while (i < keyValues.size) { - val left = keyValues(i) - val right = other.keyValues(i) - val curDirection = directions(i) - val curDataType = dataTypes(i) - - logger.debug(s"Comparing $left, $right as $curDataType order $curDirection") - // TODO: Use numeric here too? - val comparison = - if (left == null && right == null) { - 0 - } else if (left == null) { - if (curDirection == Ascending) -1 else 1 - } else if (right == null) { - if (curDirection == Ascending) 1 else -1 - } else if (curDataType == IntegerType) { - if (curDirection == Ascending) { - left.asInstanceOf[Int] compare right.asInstanceOf[Int] - } else { - right.asInstanceOf[Int] compare left.asInstanceOf[Int] - } - } else if (curDataType == DoubleType) { - if (curDirection == Ascending) { - left.asInstanceOf[Double] compare right.asInstanceOf[Double] - } else { - right.asInstanceOf[Double] compare left.asInstanceOf[Double] - } - } else if (curDataType == LongType) { - if (curDirection == Ascending) { - left.asInstanceOf[Long] compare right.asInstanceOf[Long] - } else { - right.asInstanceOf[Long] compare left.asInstanceOf[Long] - } - } else if (curDataType == StringType) { - if (curDirection == Ascending) { - left.asInstanceOf[String] compare right.asInstanceOf[String] - } else { - right.asInstanceOf[String] compare left.asInstanceOf[String] - } - } else { - sys.error(s"Comparison not yet implemented for: $curDataType") - } - - if (comparison != 0) return comparison - i += 1 - } - return 0 - } - } - - val rdd = child.execute().map { row => - val input = Vector(row) - val sortKey = new SortKey( - sortingExpressions.map(s => Evaluate(s.child, input)).toIndexedSeq) - - (sortKey, row) - } + val rdd = child.execute().map(r => (r,null)) val part = new RangePartitioner(width, rdd, ascending = true) - val shuffled = new ShuffledRDD[SortKey, Row, (SortKey, Row)](rdd, part) - - shuffled.map(_._2) - case _ => sys.error("Not implemented") + val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) + shuffled.map(_._1) + case _ => sys.error(s"Exchange not implemented for $newPartitioning") } } } From 330a88b7eeefa3a8c9be2c0170a6f33cb85879d6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:54:29 -0800 Subject: [PATCH 18/34] Fix bugs in AddExchange. --- .../scala/catalyst/execution/Exchange.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 1cda0a5d67a48..2966e26164ebb 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -42,6 +42,11 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) } } +/** + * Ensures that the [[catalyst.plans.physical.Partitioning Partitioning]] of input data meets the + * [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting + * [[Exchange]] Operators where required. + */ object AddExchange extends Rule[SharkPlan] { // TODO: determine the number of partitions. val numPartitions = 8 @@ -50,10 +55,14 @@ object AddExchange extends Rule[SharkPlan] { case operator: SharkPlan => def meetsRequirements = !operator.requiredChildDistribution.zip(operator.children).map { - case (required, child) => !child.outputPartitioning.satisfies(required) + case (required, child) => + val valid = child.outputPartitioning.satisfies(required) + logger.debug( + s"${if (valid) "Valid" else "Invalid"} distribution, required: $required current: ${child.outputPartitioning}") + valid }.exists(_ == false) - // TODO ASUUMES TRANSITIVITY? + // TODO ASSUMES TRANSITIVITY? def compatible = !operator.children .map(_.outputPartitioning) @@ -64,14 +73,14 @@ object AddExchange extends Rule[SharkPlan] { }.exists(_ == false) - if (false && meetsRequirements && compatible) { + if (meetsRequirements && compatible) { operator } else { val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { case (ClusteredDistribution(clustering), child) => - Exchange(HashPartitioning(clustering, 8), child) + Exchange(HashPartitioning(clustering, numPartitions), child) case (OrderedDistribution(ordering), child) => - Exchange(RangePartitioning(ordering, 8), child) + Exchange(RangePartitioning(ordering, numPartitions), child) case (UnknownDistribution, child) => child case (dist, _) => sys.error(s"Don't know how to ensure $dist") } From c0b0e60008b9bb1abbbb0a792b39ce3beb4ec444 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:54:41 -0800 Subject: [PATCH 19/34] cleanup broken doc links. --- .../plans/physical/partitioning.scala | 22 ++++++++++--------- src/main/scala/catalyst/types/dataTypes.scala | 2 +- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index eae6aedec2686..2252fe6e1c886 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -6,11 +6,12 @@ import expressions._ import types._ /** - * Specifies how tuples that share common expressions will be distributed. Distribution can - * be used to refer to two distinct physical properties: + * Specifies how tuples that share common expressions will be distributed when a query is executed + * in parallel on many machines. Distribution can be used to refer to two distinct physical + * properties: * - Inter-node partitioning of data: In this case the distribution describes how tuples are * partitioned across physical machines in a cluster. Knowing this property allows some - * operators (e.g., [[Aggregate]]) to perform partition local operations instead of global ones. + * operators (e.g., Aggregate) to perform partition local operations instead of global ones. * - Intra-partition ordering of data: In this case the distribution describes guarantees made * about how tuples are distributed within a single partition. */ @@ -28,17 +29,18 @@ case object UnknownDistribution extends Distribution case object AllTuples extends Distribution /** - * Represents data where tuples that share the same values for the `clustering` [[Expression]]s will - * be co-located. Based on the context, this can mean such tuples are either co-located in the same - * partition or they will be contiguous within a single partition. + * Represents data where tuples that share the same values for the `clustering` + * [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this + * can mean such tuples are either co-located in the same partition or they will be contiguous + * within a single partition. */ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution /** - * Represents data where tuples have been ordered according to the `ordering` [[Expression]]s. This - * is a strictly stronger guarantee than [[ClusteredDistribution]] as an ordering will ensure that - * tuples that share the same value for the ordering expressions are contiguous and will never be - * split across partitions. + * Represents data where tuples have been ordered according to the `ordering` + * [[catalyst.expressions.Expression Expressions]]. This is a strictly stronger guarantee than + * [[ClusteredDistribution]] as an ordering will ensure that tuples that share the same value for + * the ordering expressions are contiguous and will never be split across partitions. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { def clustering = ordering.map(_.child).toSet diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 86d289dc77525..2f00f499ab828 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -4,7 +4,7 @@ package types import expressions.Expression abstract class DataType { - /** Matches any expression that evaluates to this [[DataType]] */ + /** Matches any expression that evaluates to this DataType */ def unapply(a: Expression): Boolean = a match { case e: Expression if e.dataType == this => true case _ => false From 6377d0bddebe69e72070f0490d39e2b319342d66 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 14:58:33 -0800 Subject: [PATCH 20/34] Drop empty files, fix if (). --- src/main/scala/catalyst/execution/dataProperties.scala | 0 src/main/scala/catalyst/execution/distribution.scala | 0 src/main/scala/catalyst/trees/TreeNode.scala | 8 ++++---- 3 files changed, 4 insertions(+), 4 deletions(-) delete mode 100644 src/main/scala/catalyst/execution/dataProperties.scala delete mode 100644 src/main/scala/catalyst/execution/distribution.scala diff --git a/src/main/scala/catalyst/execution/dataProperties.scala b/src/main/scala/catalyst/execution/dataProperties.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/src/main/scala/catalyst/execution/distribution.scala b/src/main/scala/catalyst/execution/distribution.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index dd2085950fc3d..4e9ccd6000c69 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -91,7 +91,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val newArgs = productIterator.map { case arg: TreeNode[_] if (children contains arg) => val newChild = f(arg.asInstanceOf[BaseType]) - if(newChild fastEquals arg) + if (newChild fastEquals arg) arg else { changed = true @@ -100,7 +100,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case nonChild: AnyRef => nonChild case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** @@ -116,7 +116,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case arg: TreeNode[_] if (children contains arg) => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) - if(newChild fastEquals oldChild) + if (newChild fastEquals oldChild) oldChild else { changed = true @@ -126,7 +126,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case null => null }.toArray - if(changed) makeCopy(newArgs) else this + if (changed) makeCopy(newArgs) else this } /** From e170ccfe7193c78ad9c3caeb42c4f6f27cba7f81 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 15:17:26 -0800 Subject: [PATCH 21/34] Improve documentation and remove some spurious changes that were introduced by the merge. --- .../plans/physical/partitioning.scala | 34 +++++++++++++++---- src/main/scala/catalyst/trees/TreeNode.scala | 16 ++++----- src/main/scala/catalyst/types/dataTypes.scala | 7 +++- .../execution/HiveComparisonTest.scala | 1 - 4 files changed, 41 insertions(+), 17 deletions(-) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 2252fe6e1c886..8e973bcde9937 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -15,8 +15,7 @@ import types._ * - Intra-partition ordering of data: In this case the distribution describes guarantees made * about how tuples are distributed within a single partition. */ -sealed trait Distribution { -} +sealed trait Distribution /** * Represents a distribution where no promises are made about co-location of data. @@ -46,9 +45,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { def clustering = ordering.map(_.child).toSet } -sealed abstract trait Partitioning { - self: Product => - +sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ val width: Int @@ -58,6 +55,11 @@ sealed abstract trait Partitioning { */ def satisfies(required: Distribution): Boolean + /** + * Returns true iff all distribution guarantees made by this partitioning can also be made + * for the `other` specified partitioning. For example, [[HashPartitioning]]s are only compatible + * if the `width` of the two partitionings is the same. + */ def compatibleWith(other: Partitioning): Boolean } @@ -92,7 +94,14 @@ case object Broadcast extends Partitioning { } } -case class HashPartitioning(expressions: Seq[Expression], width: Int) extends Expression with Partitioning { +/** + * Represents a partitioning where rows are split up across partitions based on based on the hash + * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be + * in the same partition. + */ +case class HashPartitioning(expressions: Seq[Expression], width: Int) + extends Expression with Partitioning { + def children = expressions.toSeq def references = expressions.flatMap(_.references).toSet def nullable = false @@ -113,7 +122,18 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) extends Ex } } -case class RangePartitioning(ordering: Seq[SortOrder], width: Int) extends Expression with Partitioning { +/** + * Represents a partitioning where rows are split across partitions based on some total ordering of + * the expressions specified in `ordering`. When data is partitioned in this manner the following + * two conditions are guaranteed to hold: + * - All row where the expressions in `ordering` evaluate to the same values will be in the same + * partition. + * - Each partition will have a `min` and `max` row, relative to the given ordering. All rows + * that are in between `min` and `max` in this `ordering` will reside in this partition. + */ +case class RangePartitioning(ordering: Seq[SortOrder], width: Int) + extends Expression with Partitioning { + def children = ordering.toSeq def references = ordering.flatMap(_.references).toSet def nullable = false diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index 4e9ccd6000c69..b9599dc9301d6 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -89,11 +89,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def mapChildren(f: BaseType => BaseType): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = f(arg.asInstanceOf[BaseType]) - if (newChild fastEquals arg) + if (newChild fastEquals arg) { arg - else { + } else { changed = true newChild } @@ -113,12 +113,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) - if (newChild fastEquals oldChild) + if (newChild fastEquals oldChild) { oldChild - else { + } else { changed = true newChild } @@ -207,7 +207,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildrenUp(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { changed = true @@ -217,7 +217,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } case m: Map[_,_] => m case args: Traversable[_] => args.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) if (!(newChild fastEquals arg)) { changed = true diff --git a/src/main/scala/catalyst/types/dataTypes.scala b/src/main/scala/catalyst/types/dataTypes.scala index 2f00f499ab828..6e864d047ff4a 100644 --- a/src/main/scala/catalyst/types/dataTypes.scala +++ b/src/main/scala/catalyst/types/dataTypes.scala @@ -31,7 +31,11 @@ case object BooleanType extends NativeType { } abstract class NumericType extends NativeType { - // Unfortunately we can't get this implicitly as that breaks Spark Serialization. + // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for + // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a + // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets + // desugared by the compiler into an argument to the objects constructor. This means there is no + // longer an no argument constructor and thus the JVM cannot serialize the object anymore. val numeric: Numeric[JvmType] } @@ -42,6 +46,7 @@ object IntegralType { case _ => false } } + abstract class IntegralType extends NumericType { val integral: Integral[JvmType] } diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index c98b2ebb50f6e..830ce7a88395e 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -8,7 +8,6 @@ import frontend.hive.{ExplainCommand, Command} import plans.physical._ import util._ - /** * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. From 167162feea43ba2e07d2a49ab66b640d62c47205 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 15:28:35 -0800 Subject: [PATCH 22/34] more merge errors, cleanup. --- src/main/scala/catalyst/execution/SharkPlan.scala | 6 +++--- src/main/scala/catalyst/expressions/Evaluate.scala | 13 ++++++++----- .../scala/catalyst/expressions/predicates.scala | 11 +++++++---- src/main/scala/catalyst/trees/TreeNode.scala | 4 ++-- .../apache/spark/rdd/SharkPairRDDFunctions.scala | 7 ++++--- 5 files changed, 24 insertions(+), 17 deletions(-) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index ba34a3c178e77..0f5617746f41b 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -3,15 +3,15 @@ package execution import org.apache.spark.rdd.RDD -import plans.QueryPlan -import plans.physical._ +import catalyst.plans.QueryPlan +import catalyst.plans.physical._ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { self: Product => // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ - def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG! + def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! /** Specifies any partition requirements on the input data for this operator. */ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index b563041590c02..c5a75d1412ea2 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -34,7 +34,7 @@ object Evaluate extends Logging { @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -52,7 +52,7 @@ object Evaluate extends Logging { @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -69,7 +69,8 @@ object Evaluate extends Logging { @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + if (e1.dataType != e2.dataType) + throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) if (evalE1 == null || evalE2 == null) @@ -112,7 +113,7 @@ object Evaluate extends Logging { case UnaryMinus(child) => n1(child, _.negate(_)) /* Control Flow */ - case If(e, t, f) => if(eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) + case If(e, t, f) => if (eval(e).asInstanceOf[Boolean]) eval(t) else eval(f) /* Comparisons */ case Equals(l, r) => @@ -226,7 +227,9 @@ object Evaluate extends Logging { case implementedFunction: ImplementedUdf => implementedFunction.evaluate(implementedFunction.children.map(eval)) - case a: Attribute => throw new OptimizationException(a, "Unable to evaluate unbound reference without access to the input schema.") + case a: Attribute => + throw new OptimizationException(a, + "Unable to evaluate unbound reference without access to the input schema.") case other => throw new OptimizationException(other, "evaluation not implemented") } diff --git a/src/main/scala/catalyst/expressions/predicates.scala b/src/main/scala/catalyst/expressions/predicates.scala index f21c1cafc0e08..b8d578dc1c546 100644 --- a/src/main/scala/catalyst/expressions/predicates.scala +++ b/src/main/scala/catalyst/expressions/predicates.scala @@ -83,15 +83,18 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E def nullable = false } -case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) extends Expression { +case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) + extends Expression { + def children = predicate :: trueValue :: falseValue :: Nil - def nullable = children.exists(_.nullable) + def nullable = trueValue.nullable || falseValue.nullable def references = children.flatMap(_.references).toSet override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { - if (!resolved) + if (!resolved) { throw new UnresolvedException( - this, s"datatype. Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + this, s"Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + } trueValue.dataType } } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index b9599dc9301d6..e45dfe55b0ebb 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -163,7 +163,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformChildrenDown(rule: PartialFunction[BaseType, BaseType]): this.type = { var changed = false val newArgs = productIterator.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { changed = true @@ -173,7 +173,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } case m: Map[_,_] => m case args: Traversable[_] => args.map { - case arg: TreeNode[_] if (children contains arg) => + case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) if (!(newChild fastEquals arg)) { changed = true diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala index d0b0261895208..5f430a223f0c3 100644 --- a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -1,12 +1,13 @@ package org.apache.spark.rdd +import scala.language.implicitConversions + import scala.reflect._ -import org.apache.spark._ import scala.collection.mutable.ArrayBuffer + +import org.apache.spark._ import org.apache.spark.Aggregator import org.apache.spark.SparkContext._ - -import scala.language.implicitConversions import org.apache.spark.util.collection.AppendOnlyMap /** From 42859620fdc5129e66a6eaaf613ab67bf5fe6375 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 23 Jan 2014 15:33:41 -0800 Subject: [PATCH 23/34] Remove temporary test cases --- .../catalyst/execution/DslQueryTests1.scala | 74 ------------------- 1 file changed, 74 deletions(-) delete mode 100644 src/test/scala/catalyst/execution/DslQueryTests1.scala diff --git a/src/test/scala/catalyst/execution/DslQueryTests1.scala b/src/test/scala/catalyst/execution/DslQueryTests1.scala deleted file mode 100644 index dbce6a9743ba4..0000000000000 --- a/src/test/scala/catalyst/execution/DslQueryTests1.scala +++ /dev/null @@ -1,74 +0,0 @@ -package catalyst -package execution - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import analysis._ -import expressions._ -import plans._ -import plans.logical.LogicalPlan -import types._ - -/* Implicits */ -import dsl._ - -class DslQueryTests1 extends FunSuite with BeforeAndAfterAll { - override def beforeAll() { - // By clearing the port we force Spark to pick a new one. This allows us to rerun tests - // without restarting the JVM. - System.clearProperty("spark.driver.port") - System.clearProperty("spark.hostPort") - } - - import TestShark._ - import TestData._ - - test("inner join and then group by using the same column") { - val x = testData2.subquery('x) - val y = testData3.subquery('y) - checkAnswer( - x.join(y).where("x.a".attr === "y.a".attr) - .groupBy("x.a".attr)("x.a".attr, Count("x.b".attr)), - (1,2) :: - (2,2) :: Nil - ) - } - - /** - * Runs the plan and makes sure the answer matches the expected result. - * @param plan the query to be executed - * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. - */ - protected def checkAnswer(plan: LogicalPlan, expectedAnswer: Any): Unit = { - val convertedAnswer = expectedAnswer match { - case s: Seq[_] if s.isEmpty => s - case s: Seq[_] if s.head.isInstanceOf[Product] && - !s.head.isInstanceOf[Seq[_]] => s.map(_.asInstanceOf[Product].productIterator.toIndexedSeq) - case s: Seq[_] => s - case singleItem => Seq(Seq(singleItem)) - } - - val isSorted = plan.collect { case s: logical.Sort => s}.nonEmpty - def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer - val sharkAnswer = try plan.toRdd.collect().toSeq catch { - case e: Exception => - fail( - s""" - |Exception thrown while executing query: - |$plan - |== Physical Plan == - |${plan.executedPlan} - |== Exception == - |$e - """.stripMargin) - } - println( - s""" - |Logical plan: - |$plan - |== Physical Plan == - |${plan.executedPlan} - """.stripMargin) - assert(prepareAnswer(convertedAnswer) === prepareAnswer(sharkAnswer)) - } -} \ No newline at end of file From 548e479883c6da18e939ac87862ee04d14242797 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 25 Jan 2014 22:05:23 +0800 Subject: [PATCH 24/34] merge master into exchangeOperator and fix code style --- build.sbt | 4 ++++ src/main/scala/catalyst/execution/Exchange.scala | 6 +++--- .../scala/catalyst/execution/basicOperators.scala | 8 +++----- src/main/scala/catalyst/execution/joins.scala | 13 +++++++------ .../catalyst/execution/planningStrategies.scala | 4 ++-- .../scala/catalyst/expressions/Evaluate.scala | 3 ++- src/main/scala/catalyst/planning/patterns.scala | 15 +++++++++++++++ .../catalyst/plans/physical/partitioning.scala | 14 ++++++++++---- .../apache/spark/rdd/SharkPairRDDFunctions.scala | 6 ++++-- .../catalyst/execution/HiveComparisonTest.scala | 14 +++++++++++--- 10 files changed, 61 insertions(+), 26 deletions(-) diff --git a/build.sbt b/build.sbt index 28e9967946e3b..a1a52867305e7 100644 --- a/build.sbt +++ b/build.sbt @@ -75,4 +75,8 @@ mergeStrategy in assembly := { case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first +} + +scalacOptions in (Compile, doc) <++= (baseDirectory) map { + bd => Seq("-sourcepath", bd.getAbsolutePath, "-doc-source-url","https://github.com/databricks/catalyst/blob/master/€{FILE_PATH}.scala") } \ No newline at end of file diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 2966e26164ebb..2238d49c33a39 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -11,7 +11,7 @@ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD case class Exchange(newPartitioning: Partitioning, child: SharkPlan) - extends UnaryNode { + extends UnaryNode { override def outputPartitioning = newPartitioning def output = child.output @@ -30,7 +30,6 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) case RangePartitioning(sortingExpressions, width) => // TODO: ShuffledRDD should take an Ordering. - import scala.math.Ordering.Implicits._ implicit val ordering = new RowOrdering(sortingExpressions) val rdd = child.execute().map(r => (r,null)) @@ -58,7 +57,8 @@ object AddExchange extends Rule[SharkPlan] { case (required, child) => val valid = child.outputPartitioning.satisfies(required) logger.debug( - s"${if (valid) "Valid" else "Invalid"} distribution, required: $required current: ${child.outputPartitioning}") + s"${if (valid) "Valid" else "Invalid"} distribution," + + s"required: $required current: ${child.outputPartitioning}") valid }.exists(_ == false) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 1fc81dd665a39..014aae7110676 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -28,12 +28,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: def execute() = child.execute().sample(withReplacement, fraction, seed) } -case class Union(left: SharkPlan, right: SharkPlan)(@transient sc: SharkContext) - extends BinaryNode { +case class Union(children: Seq[SharkPlan])(@transient sc: SharkContext) extends SharkPlan { // TODO: attributes output by union should be distinct for nullability purposes - def output = left.output - // TODO: is it more efficient to union a bunch of rdds at once? should union be variadic? - def execute() = sc.union(left.execute(), right.execute()) + def output = children.head.output + def execute() = sc.union(children.map(_.execute())) override def otherCopyArgs = sc :: Nil } diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 380f64dee8753..ef2d6205f2b40 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -18,22 +18,23 @@ case class SparkEquiInnerJoin( left: SharkPlan, right: SharkPlan) extends BinaryNode { - - override def requiredChildDistribution= + override def requiredChildDistribution = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil def output = left.output ++ right.output def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute.map { row => + val leftWithKeys = left.execute().map { row => val joinKeys = leftKeys.map(Evaluate(_, Vector(row))) - logger.debug(s"Generated left join keys [${leftKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") + logger.debug(s"Generated left join keys [${leftKeys.mkString(",")}] =>" + + s"[${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } - val rightWithKeys = right.execute.map { row => + val rightWithKeys = right.execute().map { row => val joinKeys = rightKeys.map(Evaluate(_, Vector(EmptyRow, row))) - logger.debug(s"Generated right join keys [${rightKeys.mkString(",")}] => [${joinKeys.mkString(",")}] given row $row") + logger.debug(s"Generated right join keys [${rightKeys.mkString(",")}] =>" + + s"[${joinKeys.mkString(",")}] given row $row") (joinKeys, row) } diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index d20cbb5d70174..959115b1278e2 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -130,8 +130,8 @@ trait PlanningStrategies { execution.LocalRelation(output, data.map(_.productIterator.toVector))(sc) :: Nil case logical.StopAfter(limit, child) => execution.StopAfter(Evaluate(limit, Nil).asInstanceOf[Int], planLater(child))(sc) :: Nil - case logical.Union(left, right) => - execution.Union(planLater(left), planLater(right))(sc) :: Nil + case Unions(unionChildren) => + execution.Union(unionChildren.map(planLater))(sc) :: Nil case logical.Transform(input, script, output, child) => execution.Transform(input, script, output, planLater(child))(sc) :: Nil case _ => Nil diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index c5a75d1412ea2..efeceaa42d305 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -26,7 +26,8 @@ object Evaluate extends Logging { else e.dataType match { case n: NumericType => - f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType](n.numeric, eval(e).asInstanceOf[n.JvmType]) + val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] + castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } diff --git a/src/main/scala/catalyst/planning/patterns.scala b/src/main/scala/catalyst/planning/patterns.scala index 670923b436d1c..147a8fc6dffc9 100644 --- a/src/main/scala/catalyst/planning/patterns.scala +++ b/src/main/scala/catalyst/planning/patterns.scala @@ -25,4 +25,19 @@ object FilteredOperation extends PredicateHelper { collectFilters(filters ++ splitConjunctivePredicates(condition), child) case other => (filters, other) } +} + +/** + * A pattern that collects all adjacent unions and returns their children as a Seq. + */ +object Unions { + def unapply(plan: LogicalPlan): Option[Seq[LogicalPlan]] = plan match { + case u: Union => Some(collectUnionChildren(u)) + case _ => None + } + + private def collectUnionChildren(plan: LogicalPlan): Seq[LogicalPlan] = plan match { + case Union(l, r) => collectUnionChildren(l) ++ collectUnionChildren(r) + case other => other :: Nil + } } \ No newline at end of file diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 8e973bcde9937..1fe9f6a4306a6 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -68,6 +68,7 @@ case class UnknownPartitioning(width: Int) extends Partitioning { case UnknownDistribution => true case _ => false } + def compatibleWith(other: Partitioning): Boolean = other match { case UnknownPartitioning(_) => true case _ => false @@ -78,6 +79,7 @@ case object Unpartitioned extends Partitioning { val width = 1 override def satisfies(required: Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { case Unpartitioned => true case _ => false @@ -87,7 +89,8 @@ case object Unpartitioned extends Partitioning { case object Broadcast extends Partitioning { val width = 1 - override def satisfies(required:Distribution): Boolean = true + override def satisfies(required: Distribution): Boolean = true + override def compatibleWith(other: Partitioning) = other match { case Unpartitioned => true case _ => false @@ -95,12 +98,13 @@ case object Broadcast extends Partitioning { } /** - * Represents a partitioning where rows are split up across partitions based on based on the hash + * Represents a partitioning where rows are split up across partitions based on the hash * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be * in the same partition. */ case class HashPartitioning(expressions: Seq[Expression], width: Int) - extends Expression with Partitioning { + extends Expression + with Partitioning { def children = expressions.toSeq def references = expressions.flatMap(_.references).toSet @@ -115,6 +119,7 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) clusteringSet.subsetOf(requiredClustering.toSet) case _ => false } + override def compatibleWith(other: Partitioning) = other match { case Broadcast => true case h: HashPartitioning if h == this => true @@ -132,7 +137,8 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) * that are in between `min` and `max` in this `ordering` will reside in this partition. */ case class RangePartitioning(ordering: Seq[SortOrder], width: Int) - extends Expression with Partitioning { + extends Expression + with Partitioning { def children = ordering.toSeq def references = ordering.flatMap(_.references).toSet diff --git a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala index 5f430a223f0c3..1599bb51cda4e 100644 --- a/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala +++ b/src/main/scala/org/apache/spark/rdd/SharkPairRDDFunctions.scala @@ -11,8 +11,10 @@ import org.apache.spark.SparkContext._ import org.apache.spark.util.collection.AppendOnlyMap /** - * Extra functions for Shark available on RDDs of (key, value) pairs through an implicit conversion. - * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to use these functions. + * Extra functions for Shark available on RDDs of (key, value) pairs through + * an implicit conversion. + * Import `org.apache.spark.SharkPairRDDFunctions._` at the top of your program to + * use these functions. */ class SharkPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Logging diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 830ce7a88395e..1b4d01c3ce3a9 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -87,7 +87,11 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G hiveFailedDirectory.mkdir() // Not atomic! /** All directories that contain per-query output files */ - val outputDirectories = Seq(passedDirectory, failedDirectory, wrongDirectory, hiveFailedDirectory) + val outputDirectories = Seq( + passedDirectory, + failedDirectory, + wrongDirectory, + hiveFailedDirectory) protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { @@ -96,13 +100,17 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G new java.math.BigInteger(1, digest.digest).toString(16) } - protected def prepareAnswer(sharkQuery: TestShark.type#SharkSqlQuery, answer: Seq[String]): Seq[String] = { + protected def prepareAnswer( + sharkQuery: TestShark.type#SharkSqlQuery, + answer: Seq[String]): Seq[String] = { val orderedAnswer = sharkQuery.parsed match { // Clean out non-deterministic time schema info. case _: Command => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _ => // TODO: Really we only care about the final total ordering here... - val isOrdered = sharkQuery.executedPlan.collect { case s @ Exchange(r: RangePartitioning, _) => s}.nonEmpty + val isOrdered = sharkQuery.executedPlan.collect { + case s @ Exchange(r: RangePartitioning, _) => s + }.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer } From 2957f31b9aa558423df25bb730ea9fe2d24699ee Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 26 Jan 2014 20:55:51 +0800 Subject: [PATCH 25/34] addressed comments on PR --- .../scala/catalyst/analysis/unresolved.scala | 2 +- src/main/scala/catalyst/errors/package.scala | 11 ++--- .../scala/catalyst/execution/Exchange.scala | 17 ++++--- .../scala/catalyst/execution/SharkPlan.scala | 2 +- .../catalyst/execution/basicOperators.scala | 1 + .../scala/catalyst/expressions/Evaluate.scala | 12 ++--- .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../plans/physical/partitioning.scala | 45 ++++++++++--------- src/main/scala/catalyst/trees/TreeNode.scala | 2 +- .../execution/DistributionTests.scala | 8 ++-- 10 files changed, 55 insertions(+), 47 deletions(-) diff --git a/src/main/scala/catalyst/analysis/unresolved.scala b/src/main/scala/catalyst/analysis/unresolved.scala index 54ed4d9802392..d198c6dd8ce1c 100644 --- a/src/main/scala/catalyst/analysis/unresolved.scala +++ b/src/main/scala/catalyst/analysis/unresolved.scala @@ -10,7 +10,7 @@ import trees.TreeNode * resolved. */ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String) extends - errors.OptimizationException(tree, s"Invalid call to $function on unresolved object") + errors.TreeNodeException(tree, s"Invalid call to $function on unresolved object") /** * Holds the name of a relation that has yet to be looked up in a [[Catalog]]. diff --git a/src/main/scala/catalyst/errors/package.scala b/src/main/scala/catalyst/errors/package.scala index 41bd9c1d8055e..0f24fa5ad9f43 100644 --- a/src/main/scala/catalyst/errors/package.scala +++ b/src/main/scala/catalyst/errors/package.scala @@ -7,7 +7,7 @@ import trees._ */ package object errors { - class OptimizationException[TreeType <: TreeNode[_]] + class TreeNodeException[TreeType <: TreeNode[_]] (tree: TreeType, msg: String, cause: Throwable = null) extends Exception(msg, cause) { override def getMessage: String = { @@ -17,17 +17,18 @@ package object errors { } /** - * Wraps any exceptions that are thrown while executing `f` in an [[OptimizationException]], - * attaching the provided `tree`. + * Wraps any exceptions that are thrown while executing `f` in a + * [[catalyst.errors.TreeNodeException TreeNodeException]], attaching the provided `tree`. */ def attachTree[TreeType <: TreeNode[_], A](tree: TreeType, msg: String = "")(f: => A): A = { try f catch { - case e: Exception => throw new OptimizationException(tree, msg, e) + case e: Exception => throw new TreeNodeException(tree, msg, e) } } /** - * Executes `f` which is expected to throw an OptimizationException. The first tree encountered in + * Executes `f` which is expected to throw a + * [[catalyst.errors.TreeNodeException TreeNodeException]]. The first tree encountered in * the stack of exceptions of type `TreeType` is returned. */ def getTree[TreeType <: TreeNode[_]](f: => Unit): TreeType = ??? // TODO: Implement diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 2238d49c33a39..698167e16cb7c 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -18,25 +18,26 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) def execute() = attachTree(this , "execute") { newPartitioning match { - case HashPartitioning(expressions, width) => + case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) } - val part = new HashPartitioner(width) + val part = new HashPartitioner(numPartitions) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - case RangePartitioning(sortingExpressions, width) => + case RangePartitioning(sortingExpressions, numPartitions) => // TODO: ShuffledRDD should take an Ordering. implicit val ordering = new RowOrdering(sortingExpressions) val rdd = child.execute().map(r => (r,null)) - val part = new RangePartitioner(width, rdd, ascending = true) + val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) shuffled.map(_._1) case _ => sys.error(s"Exchange not implemented for $newPartitioning") + // TODO: Handle SinglePartition and BroadcastPartitioning. } } } @@ -47,7 +48,7 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) * [[Exchange]] Operators where required. */ object AddExchange extends Rule[SharkPlan] { - // TODO: determine the number of partitions. + // TODO: Determine the number of partitions. val numPartitions = 8 def apply(plan: SharkPlan): SharkPlan = plan.transformUp { @@ -62,7 +63,7 @@ object AddExchange extends Rule[SharkPlan] { valid }.exists(_ == false) - // TODO ASSUMES TRANSITIVITY? + // TODO: ASSUMES TRANSITIVITY? def compatible = !operator.children .map(_.outputPartitioning) @@ -76,12 +77,14 @@ object AddExchange extends Rule[SharkPlan] { if (meetsRequirements && compatible) { operator } else { + // TODO: It is possible that only a child does not meet requirement. val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { case (ClusteredDistribution(clustering), child) => Exchange(HashPartitioning(clustering, numPartitions), child) case (OrderedDistribution(ordering), child) => Exchange(RangePartitioning(ordering, numPartitions), child) - case (UnknownDistribution, child) => child + case (UnspecifiedDistribution, child) => child + // TODO: Handle AllTuples. One possible way is to use Exchange(HashPartitioning(Nil, 1), child). case (dist, _) => sys.error(s"Don't know how to ensure $dist") } operator.withNewChildren(repartitionedChildren) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 0f5617746f41b..6fb7d870454a0 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -13,7 +13,7 @@ abstract class SharkPlan extends QueryPlan[SharkPlan] with Logging { /** Specifies how data is partitioned across different nodes in the cluster. */ def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! /** Specifies any partition requirements on the input data for this operator. */ - def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnknownDistribution) + def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) /** * Runs this query returning the result as an RDD. diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 014aae7110676..1a25407ff6508 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -53,6 +53,7 @@ case class SortPartitions(sortOrder: Seq[SortOrder], child: SharkPlan) extends U lazy val ordering = new RowOrdering(sortOrder) def execute() = attachTree(this, "sort") { + // TODO: Optimize sorting operation? child.execute() .mapPartitions( iterator => iterator.toArray.sorted(ordering).iterator, diff --git a/src/main/scala/catalyst/expressions/Evaluate.scala b/src/main/scala/catalyst/expressions/Evaluate.scala index efeceaa42d305..febef1658c209 100644 --- a/src/main/scala/catalyst/expressions/Evaluate.scala +++ b/src/main/scala/catalyst/expressions/Evaluate.scala @@ -35,7 +35,7 @@ object Evaluate extends Logging { @inline def n2(e1: Expression, e2: Expression, f: ((Numeric[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new TreeNodeException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -53,7 +53,7 @@ object Evaluate extends Logging { @inline def f2(e1: Expression, e2: Expression, f: ((Fractional[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new TreeNodeException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) @@ -71,7 +71,7 @@ object Evaluate extends Logging { @inline def i2(e1: Expression, e2: Expression, f: ((Integral[Any], Any, Any) => Any)): Any = { if (e1.dataType != e2.dataType) - throw new OptimizationException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") + throw new TreeNodeException(e, s"Data types do not match ${e1.dataType} != ${e2.dataType}") val evalE1 = eval(e1) val evalE2 = eval(e2) if (evalE1 == null || evalE2 == null) @@ -218,7 +218,7 @@ object Evaluate extends Logging { /* References to input tuples */ case br @ BoundReference(inputTuple, ordinal, _) => try input(inputTuple)(ordinal) catch { case iob: IndexOutOfBoundsException => - throw new OptimizationException(br, s"Reference not in tuple: $input") + throw new TreeNodeException(br, s"Reference not in tuple: $input") } /* Functions */ @@ -229,9 +229,9 @@ object Evaluate extends Logging { implementedFunction.evaluate(implementedFunction.children.map(eval)) case a: Attribute => - throw new OptimizationException(a, + throw new TreeNodeException(a, "Unable to evaluate unbound reference without access to the input schema.") - case other => throw new OptimizationException(other, "evaluation not implemented") + case other => throw new TreeNodeException(other, "evaluation not implemented") } logger.debug(s"Evaluated $e => $result of type ${if (result == null) "null" else result.getClass.getName}, expected: ${e.dataType}") diff --git a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala index 792eb88e77a28..87ef2319cc2f1 100644 --- a/src/main/scala/catalyst/plans/logical/LogicalPlan.scala +++ b/src/main/scala/catalyst/plans/logical/LogicalPlan.scala @@ -45,7 +45,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { case a :: Nil => Some(a) // One match, use it. case Nil => None // No matches. case ambiguousReferences => - throw new OptimizationException( + throw new TreeNodeException( this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") } } diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 1fe9f6a4306a6..2ede02f64048a 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -20,10 +20,11 @@ sealed trait Distribution /** * Represents a distribution where no promises are made about co-location of data. */ -case object UnknownDistribution extends Distribution +case object UnspecifiedDistribution extends Distribution /** - * Represents a distribution where a single operation can observe all tuples in the dataset. + * Represents a distribution that only has a single partition and all tuples of the dataset + * are co-located. */ case object AllTuples extends Distribution @@ -47,25 +48,27 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ - val width: Int + val numPartitions: Int /** - * Returns true iff the guarantees made by this [[Distribution]] are sufficient to satisfy all - * guarantees mandated by the `required` distribution. + * Returns true iff the guarantees made by this + * [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy all + * guarantees mandated by the `required` [[catalyst.plans.physical.Distribution Distribution]]. */ def satisfies(required: Distribution): Boolean /** * Returns true iff all distribution guarantees made by this partitioning can also be made - * for the `other` specified partitioning. For example, [[HashPartitioning]]s are only compatible - * if the `width` of the two partitionings is the same. + * for the `other` specified partitioning. + * For example, two [[catalyst.plans.physical.HashPartitioning HashPartitioning]]s are + * only compatible if the `numPartitions` of them is the same. */ def compatibleWith(other: Partitioning): Boolean } -case class UnknownPartitioning(width: Int) extends Partitioning { +case class UnknownPartitioning(numPartitions: Int) extends Partitioning { def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true + case UnspecifiedDistribution => true case _ => false } @@ -75,24 +78,24 @@ case class UnknownPartitioning(width: Int) extends Partitioning { } } -case object Unpartitioned extends Partitioning { - val width = 1 +case object SinglePartition extends Partitioning { + val numPartitions = 1 override def satisfies(required: Distribution): Boolean = true override def compatibleWith(other: Partitioning) = other match { - case Unpartitioned => true + case SinglePartition => true case _ => false } } -case object Broadcast extends Partitioning { - val width = 1 +case object BroadcastPartitioning extends Partitioning { + val numPartitions = 1 override def satisfies(required: Distribution): Boolean = true override def compatibleWith(other: Partitioning) = other match { - case Unpartitioned => true + case SinglePartition => true case _ => false } } @@ -102,7 +105,7 @@ case object Broadcast extends Partitioning { * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be * in the same partition. */ -case class HashPartitioning(expressions: Seq[Expression], width: Int) +case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { @@ -114,14 +117,14 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) lazy val clusteringSet = expressions.toSet def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true + case UnspecifiedDistribution => true case ClusteredDistribution(requiredClustering) => clusteringSet.subsetOf(requiredClustering.toSet) case _ => false } override def compatibleWith(other: Partitioning) = other match { - case Broadcast => true + case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false } @@ -136,7 +139,7 @@ case class HashPartitioning(expressions: Seq[Expression], width: Int) * - Each partition will have a `min` and `max` row, relative to the given ordering. All rows * that are in between `min` and `max` in this `ordering` will reside in this partition. */ -case class RangePartitioning(ordering: Seq[SortOrder], width: Int) +case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { @@ -148,7 +151,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], width: Int) lazy val clusteringSet = ordering.map(_.child).toSet def satisfies(required: Distribution): Boolean = required match { - case UnknownDistribution => true + case UnspecifiedDistribution => true case OrderedDistribution(requiredOrdering) => val minSize = Seq(requiredOrdering.size, ordering.size).min requiredOrdering.take(minSize) == ordering.take(minSize) @@ -158,7 +161,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], width: Int) } def compatibleWith(other: Partitioning) = other match { - case Broadcast => true + case BroadcastPartitioning => true case r: RangePartitioning if r == this => true case _ => false } diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index e45dfe55b0ebb..ccfbc7237a564 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -256,7 +256,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } } catch { case e: java.lang.IllegalArgumentException => - throw new OptimizationException( + throw new TreeNodeException( this, s"Failed to copy node. Is otherCopyArgs specified correctly for $nodeName?") } } diff --git a/src/test/scala/catalyst/execution/DistributionTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala index 61e074127d05b..db37a8e5aa30b 100644 --- a/src/test/scala/catalyst/execution/DistributionTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -28,7 +28,7 @@ class DistributionTests extends FunSuite { // Cases which do not need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - UnknownDistribution, + UnspecifiedDistribution, true) checkSatisfied( @@ -42,12 +42,12 @@ class DistributionTests extends FunSuite { true) checkSatisfied( - Unpartitioned, + SinglePartition, ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( - Unpartitioned, + SinglePartition, OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) @@ -90,7 +90,7 @@ class DistributionTests extends FunSuite { // Cases which do not need an exchange between two data properties. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - UnknownDistribution, + UnspecifiedDistribution, true) checkSatisfied( From 9fb357ab90bca4d8b801983b952ff99085eb11f9 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 08:55:35 +0800 Subject: [PATCH 26/34] use getSpecifiedDistribution to create Distribution. ClusteredDistribution and OrderedDistribution do not take Nil as inptu expressions. --- .../scala/catalyst/execution/aggregates.scala | 5 +- src/main/scala/catalyst/execution/joins.scala | 14 ++++-- .../plans/physical/partitioning.scala | 46 ++++++++++++++--- .../execution/DistributionTests.scala | 49 ++++++++++--------- 4 files changed, 74 insertions(+), 40 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index 40925bd6bab1b..d7fcd3d25d46a 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,11 +3,10 @@ package execution import catalyst.errors._ import catalyst.expressions._ -import catalyst.plans.physical._ +import catalyst.plans.physical.Distribution._ import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ -import org.apache.spark.SparkContext._ import SharkPairRDDFunctions._ case class Aggregate( @@ -16,7 +15,7 @@ case class Aggregate( child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { - override def requiredChildDistribution = Seq(ClusteredDistribution(groupingExpressions)) + override def requiredChildDistribution = getSpecifiedDistribution(groupingExpressions) :: Nil override def otherCopyArgs = sc :: Nil def output = aggregateExpressions.map(_.toAttribute) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index ef2d6205f2b40..267b020dfc986 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -2,24 +2,27 @@ package catalyst package execution import scala.collection.mutable - -import org.apache.spark.rdd.RDD +import scala.Some import errors._ import expressions._ import plans._ -import plans.physical._ - +import catalyst.plans.physical.Distribution._ import org.apache.spark.rdd.SharkPairRDDFunctions._ +import org.apache.spark.rdd.RDD +import catalyst.plans.physical.Partitioning + case class SparkEquiInnerJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: SharkPlan, right: SharkPlan) extends BinaryNode { + override def outputPartitioning: Partitioning = left.outputPartitioning + override def requiredChildDistribution = - ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + getSpecifiedDistribution(leftKeys) :: getSpecifiedDistribution(rightKeys) :: Nil def output = left.output ++ right.output @@ -66,6 +69,7 @@ case class BroadcastNestedLoopJoin( streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SharkContext) extends BinaryNode { + // TODO: Override requiredChildDistribution, outputPartitioning, and use Exchange to broadcast override def otherCopyArgs = sc :: Nil diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index 2ede02f64048a..dbcbb4fd439bf 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -28,13 +28,20 @@ case object UnspecifiedDistribution extends Distribution */ case object AllTuples extends Distribution +// TODO: Add a BroadcastDistribution? /** * Represents data where tuples that share the same values for the `clustering` * [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this * can mean such tuples are either co-located in the same partition or they will be contiguous * within a single partition. */ -case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution +case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { + if (clustering == Nil) { + throw new IllegalArgumentException("The clustering expressions of a ClusteredDistribution " + + "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + "a single partition.") + } +} /** * Represents data where tuples have been ordered according to the `ordering` @@ -43,17 +50,40 @@ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distributi * the ordering expressions are contiguous and will never be split across partitions. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { + if (ordering == Nil) { + throw new IllegalArgumentException("The ordering expressions of a OrderedDistribution " + + "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + "a single partition.") + } + def clustering = ordering.map(_.child).toSet } +object Distribution { + def getSpecifiedDistribution(expressions: Seq[Expression]): Distribution = { + if (expressions == Nil) { + AllTuples + } else { + if (expressions.forall(exp => exp.isInstanceOf[SortOrder])) { + OrderedDistribution(expressions.asInstanceOf[Seq[SortOrder]]) + } else { + ClusteredDistribution(expressions) + } + } + } +} + sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ val numPartitions: Int /** * Returns true iff the guarantees made by this - * [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy all - * guarantees mandated by the `required` [[catalyst.plans.physical.Distribution Distribution]]. + * [[catalyst.plans.physical.Partitioning Partitioning]] are sufficient to satisfy + * the partitioning scheme mandated by the `required` + * [[catalyst.plans.physical.Distribution Distribution]], i.e. the current dataset does not + * need to be re-partitioned for the `required` Distribution (it is possible that tuples within + * a partition need to be reorganized). */ def satisfies(required: Distribution): Boolean @@ -81,9 +111,9 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case object SinglePartition extends Partitioning { val numPartitions = 1 - override def satisfies(required: Distribution): Boolean = true + def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -92,9 +122,9 @@ case object SinglePartition extends Partitioning { case object BroadcastPartitioning extends Partitioning { val numPartitions = 1 - override def satisfies(required: Distribution): Boolean = true + def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -123,7 +153,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } - override def compatibleWith(other: Partitioning) = other match { + def compatibleWith(other: Partitioning) = other match { case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false diff --git a/src/test/scala/catalyst/execution/DistributionTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala index db37a8e5aa30b..0485ff6ab7bb1 100644 --- a/src/test/scala/catalyst/execution/DistributionTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -5,6 +5,7 @@ package physical import org.scalatest.FunSuite import catalyst.dsl._ +import catalyst.plans.physical.Distribution._ class DistributionTests extends FunSuite { @@ -15,16 +16,16 @@ class DistributionTests extends FunSuite { if(inputPartitioning.satisfies(requiredDistribution) != satisfied) fail( s""" - |== Input Distribution == + |== Input Partitioning == |$inputPartitioning |== Required Distribution == |$requiredDistribution - |== Does input distribution satisfy requirements? == + |== Does input partitioning satisfy required distribution? == |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) } - test("needExchange test: ClusteredDistribution is the output DataProperty") { + test("HashPartitioning is the output partitioning") { // Cases which do not need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), @@ -33,48 +34,48 @@ class DistributionTests extends FunSuite { checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) // Cases which need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('b, 'c)), + getSpecifiedDistribution(Seq('b, 'c)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('d, 'e)), + getSpecifiedDistribution(Seq('d, 'e)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - AllTuples, + getSpecifiedDistribution(Nil), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) // TODO: We should check functional dependencies @@ -86,7 +87,7 @@ class DistributionTests extends FunSuite { */ } - test("needExchange test: OrderedDistribution is the output DataProperty") { + test("RangePartitioning is the output partitioning") { // Cases which do not need an exchange between two data properties. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), @@ -95,32 +96,32 @@ class DistributionTests extends FunSuite { checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + getSpecifiedDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('c, 'b, 'a)), + getSpecifiedDistribution(Seq('c, 'b, 'a)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('b, 'c, 'a, 'd)), + getSpecifiedDistribution(Seq('b, 'c, 'a, 'd)), true) // Cases which need an exchange between two data properties. @@ -130,27 +131,27 @@ class DistributionTests extends FunSuite { // and the parallelism. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + getSpecifiedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('b.asc, 'a.asc)), + getSpecifiedDistribution(Seq('b.asc, 'a.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('a, 'b)), + getSpecifiedDistribution(Seq('a, 'b)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('c, 'd)), + getSpecifiedDistribution(Seq('c, 'd)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Nil), + getSpecifiedDistribution(Nil), false) } } \ No newline at end of file From 040fbdf02262e2ec9e02f7a789196260ad8ffc06 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 08:57:35 +0800 Subject: [PATCH 27/34] AddExchange is the only place to add Exchange operators. --- .../scala/catalyst/execution/basicOperators.scala | 11 ++++++++++- .../scala/catalyst/execution/planningStrategies.scala | 10 ++++++---- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 1a25407ff6508..487de52dc1cc4 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -3,6 +3,9 @@ package execution import catalyst.errors._ import catalyst.expressions._ +import catalyst.plans.physical.Distribution._ +import catalyst.expressions.SortOrder +import catalyst.plans.physical.{UnspecifiedDistribution, Distribution} case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -48,7 +51,13 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class SortPartitions(sortOrder: Seq[SortOrder], child: SharkPlan) extends UnaryNode { +case class SortPartitions( + sortOrder: Seq[SortOrder], + child: SharkPlan) + (override val requiredChildDistribution: Seq[Distribution] = UnspecifiedDistribution :: Nil) + extends UnaryNode { + override def otherCopyArgs = requiredChildDistribution :: Nil + @transient lazy val ordering = new RowOrdering(sortOrder) diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 959115b1278e2..558ccad8bcdc7 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -113,11 +113,13 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - // First repartition then sort locally. - execution.SortPartitions(sortExprs, - execution.Exchange(RangePartitioning(sortExprs, 8), planLater(child))) :: Nil + // Set the requiredDistribution of this SortPartitions to OrderedDistribution. + execution.SortPartitions( + sortExprs, + planLater(child))(OrderedDistribution(sortExprs) :: Nil) :: Nil case logical.SortPartitions(sortExprs, child) => - execution.SortPartitions(sortExprs, planLater(child)) :: Nil + // Set the requiredDistribution of this SortPartitions to UnspecifiedDistribution. + execution.SortPartitions(sortExprs, planLater(child))() :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => From 4802f69f4236a32d30ddf021daa31d243a472872 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 08:58:59 +0800 Subject: [PATCH 28/34] The outputPartitioning of a UnaryNode inherits its child's outputPartitioning by default. Also, update the logic in AddExchange to avoid unnecessary shuffling operations. --- .../scala/catalyst/execution/Exchange.scala | 42 ++++++++++++++----- .../scala/catalyst/execution/SharkPlan.scala | 1 + 2 files changed, 33 insertions(+), 10 deletions(-) diff --git a/src/main/scala/catalyst/execution/Exchange.scala b/src/main/scala/catalyst/execution/Exchange.scala index 698167e16cb7c..076d4a8e69a9f 100644 --- a/src/main/scala/catalyst/execution/Exchange.scala +++ b/src/main/scala/catalyst/execution/Exchange.scala @@ -5,7 +5,6 @@ import catalyst.rules.Rule import catalyst.errors._ import catalyst.expressions._ import catalyst.plans.physical._ -import catalyst.types._ import org.apache.spark.{RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD @@ -18,7 +17,7 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) def execute() = attachTree(this , "execute") { newPartitioning match { - case HashPartitioning(expressions, numPartitions) => + case HashPartitioning(expressions, numPartitions) => { // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().map { row => (buildRow(expressions.toSeq.map(Evaluate(_, Vector(row)))), row) @@ -27,17 +26,24 @@ case class Exchange(newPartitioning: Partitioning, child: SharkPlan) val shuffled = new ShuffledRDD[Row, Row, (Row, Row)](rdd, part) shuffled.map(_._2) - - case RangePartitioning(sortingExpressions, numPartitions) => + } + case RangePartitioning(sortingExpressions, numPartitions) => { // TODO: ShuffledRDD should take an Ordering. implicit val ordering = new RowOrdering(sortingExpressions) - val rdd = child.execute().map(r => (r,null)) + val rdd = child.execute().map(row => (row, null)) val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, (Row, Null)](rdd, part) + shuffled.map(_._1) + } + case SinglePartition => { + val rdd = child.execute().coalesce(1, true) + + rdd + } case _ => sys.error(s"Exchange not implemented for $newPartitioning") - // TODO: Handle SinglePartition and BroadcastPartitioning. + // TODO: Handle BroadcastPartitioning. } } } @@ -53,6 +59,8 @@ object AddExchange extends Rule[SharkPlan] { def apply(plan: SharkPlan): SharkPlan = plan.transformUp { case operator: SharkPlan => + // Check if every child's outputPartitioning satisfies the corresponding + // required data distribution. def meetsRequirements = !operator.requiredChildDistribution.zip(operator.children).map { case (required, child) => @@ -63,6 +71,13 @@ object AddExchange extends Rule[SharkPlan] { valid }.exists(_ == false) + // Check if outputPartitionings of children are compatible with each other. + // It is possible that every child satisfies its required data distribution + // but two children have incompatible outputPartitionings. For example, + // A dataset is range partitioned by "a.asc" (RangePartitioning) and another + // dataset is hash partitioned by "a" (HashPartitioning). Tuples in these two + // datasets are both clustered by "a", but these two outputPartitionings are not + // compatible. // TODO: ASSUMES TRANSITIVITY? def compatible = !operator.children @@ -73,18 +88,25 @@ object AddExchange extends Rule[SharkPlan] { case Seq(a,b) => a compatibleWith b }.exists(_ == false) + // Check if the partitioning we want to ensure is the same as the child's output + // partitioning. If so, we do not need to add the Exchange operator. + def addExchangeIfNecessary(partitioning: Partitioning, child: SharkPlan) = + if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child if (meetsRequirements && compatible) { operator } else { - // TODO: It is possible that only a child does not meet requirement. + // At least one child does not satisfies its required data distribution or + // at least one child's outputPartitioning is not compatible with another child's + // outputPartitioning. In this case, we need to add Exchange operators. val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { + case (AllTuples, child) => + addExchangeIfNecessary(SinglePartition, child) case (ClusteredDistribution(clustering), child) => - Exchange(HashPartitioning(clustering, numPartitions), child) + addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child) case (OrderedDistribution(ordering), child) => - Exchange(RangePartitioning(ordering, numPartitions), child) + addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child) case (UnspecifiedDistribution, child) => child - // TODO: Handle AllTuples. One possible way is to use Exchange(HashPartitioning(Nil, 1), child). case (dist, _) => sys.error(s"Don't know how to ensure $dist") } operator.withNewChildren(repartitionedChildren) diff --git a/src/main/scala/catalyst/execution/SharkPlan.scala b/src/main/scala/catalyst/execution/SharkPlan.scala index 6fb7d870454a0..c22ec2831fe1b 100644 --- a/src/main/scala/catalyst/execution/SharkPlan.scala +++ b/src/main/scala/catalyst/execution/SharkPlan.scala @@ -34,6 +34,7 @@ trait LeafNode extends SharkPlan with trees.LeafNode[SharkPlan] { trait UnaryNode extends SharkPlan with trees.UnaryNode[SharkPlan] { self: Product => + override def outputPartitioning: Partitioning = child.outputPartitioning } trait BinaryNode extends SharkPlan with trees.BinaryNode[SharkPlan] { From e811d1a287aa692fe4d38246230ad0620630a4cf Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 09:03:01 +0800 Subject: [PATCH 29/34] markhamstra's comments --- src/main/scala/catalyst/trees/TreeNode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/trees/TreeNode.scala b/src/main/scala/catalyst/trees/TreeNode.scala index ccfbc7237a564..814686d8936eb 100644 --- a/src/main/scala/catalyst/trees/TreeNode.scala +++ b/src/main/scala/catalyst/trees/TreeNode.scala @@ -274,7 +274,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { case tn: TreeNode[_] if children contains tn => Nil case tn: TreeNode[_] if tn.toString contains "\n" => s"(${tn.simpleString})" :: Nil case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil - case seq: Set[_] => seq.mkString("{", ",", "}") :: Nil + case set: Set[_] => set.mkString("{", ",", "}") :: Nil case other => other :: Nil }.mkString(", ") From 9ebff4776cab41553ce2376f2fb652938cbaac23 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 09:04:52 +0800 Subject: [PATCH 30/34] remove unnecessary .toSeq --- src/main/scala/catalyst/plans/physical/partitioning.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index dbcbb4fd439bf..ee382ede8a79e 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -139,7 +139,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { - def children = expressions.toSeq + def children = expressions def references = expressions.flatMap(_.references).toSet def nullable = false def dataType = IntegerType @@ -173,7 +173,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { - def children = ordering.toSeq + def children = ordering def references = ordering.flatMap(_.references).toSet def nullable = false def dataType = IntegerType From 83d452081100447b45d93e6b2228dfda130a6473 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jan 2014 11:22:03 +0800 Subject: [PATCH 31/34] marmbrus's comments --- .../scala/catalyst/execution/aggregates.scala | 10 +++- .../catalyst/execution/basicOperators.scala | 11 ++-- src/main/scala/catalyst/execution/joins.scala | 7 ++- .../execution/planningStrategies.scala | 6 +-- .../plans/physical/partitioning.scala | 51 +++++++------------ .../execution/DistributionTests.scala | 41 ++++++++------- 6 files changed, 56 insertions(+), 70 deletions(-) diff --git a/src/main/scala/catalyst/execution/aggregates.scala b/src/main/scala/catalyst/execution/aggregates.scala index d7fcd3d25d46a..f428b892de341 100644 --- a/src/main/scala/catalyst/execution/aggregates.scala +++ b/src/main/scala/catalyst/execution/aggregates.scala @@ -3,7 +3,7 @@ package execution import catalyst.errors._ import catalyst.expressions._ -import catalyst.plans.physical.Distribution._ +import catalyst.plans.physical.{ClusteredDistribution, AllTuples} import org.apache.spark.rdd.SharkPairRDDFunctions /* Implicits */ @@ -15,7 +15,13 @@ case class Aggregate( child: SharkPlan)(@transient sc: SharkContext) extends UnaryNode { - override def requiredChildDistribution = getSpecifiedDistribution(groupingExpressions) :: Nil + override def requiredChildDistribution = + if (groupingExpressions == Nil) { + AllTuples :: Nil + } else { + ClusteredDistribution(groupingExpressions) :: Nil + } + override def otherCopyArgs = sc :: Nil def output = aggregateExpressions.map(_.toAttribute) diff --git a/src/main/scala/catalyst/execution/basicOperators.scala b/src/main/scala/catalyst/execution/basicOperators.scala index 487de52dc1cc4..b44c1964fb261 100644 --- a/src/main/scala/catalyst/execution/basicOperators.scala +++ b/src/main/scala/catalyst/execution/basicOperators.scala @@ -3,9 +3,7 @@ package execution import catalyst.errors._ import catalyst.expressions._ -import catalyst.plans.physical.Distribution._ -import catalyst.expressions.SortOrder -import catalyst.plans.physical.{UnspecifiedDistribution, Distribution} +import catalyst.plans.physical.{UnspecifiedDistribution, OrderedDistribution} case class Project(projectList: Seq[NamedExpression], child: SharkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -51,12 +49,13 @@ case class StopAfter(limit: Int, child: SharkPlan)(@transient sc: SharkContext) def execute() = sc.makeRDD(executeCollect(), 1) } -case class SortPartitions( +case class Sort( sortOrder: Seq[SortOrder], + global: Boolean, child: SharkPlan) - (override val requiredChildDistribution: Seq[Distribution] = UnspecifiedDistribution :: Nil) extends UnaryNode { - override def otherCopyArgs = requiredChildDistribution :: Nil + override def requiredChildDistribution = + if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil @transient lazy val ordering = new RowOrdering(sortOrder) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index 267b020dfc986..fc9e38df7f995 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -7,11 +7,10 @@ import scala.Some import errors._ import expressions._ import plans._ -import catalyst.plans.physical.Distribution._ import org.apache.spark.rdd.SharkPairRDDFunctions._ import org.apache.spark.rdd.RDD -import catalyst.plans.physical.Partitioning +import catalyst.plans.physical.{ClusteredDistribution, Partitioning} case class SparkEquiInnerJoin( leftKeys: Seq[Expression], @@ -22,7 +21,7 @@ case class SparkEquiInnerJoin( override def outputPartitioning: Partitioning = left.outputPartitioning override def requiredChildDistribution = - getSpecifiedDistribution(leftKeys) :: getSpecifiedDistribution(rightKeys) :: Nil + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil def output = left.output ++ right.output @@ -69,7 +68,7 @@ case class BroadcastNestedLoopJoin( streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SharkContext) extends BinaryNode { - // TODO: Override requiredChildDistribution, outputPartitioning, and use Exchange to broadcast + // TODO: Override requiredChildDistribution and outputPartitioning. override def otherCopyArgs = sc :: Nil diff --git a/src/main/scala/catalyst/execution/planningStrategies.scala b/src/main/scala/catalyst/execution/planningStrategies.scala index 558ccad8bcdc7..0001c6c281ed9 100644 --- a/src/main/scala/catalyst/execution/planningStrategies.scala +++ b/src/main/scala/catalyst/execution/planningStrategies.scala @@ -114,12 +114,10 @@ trait PlanningStrategies { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => // Set the requiredDistribution of this SortPartitions to OrderedDistribution. - execution.SortPartitions( - sortExprs, - planLater(child))(OrderedDistribution(sortExprs) :: Nil) :: Nil + execution.Sort(sortExprs, true, planLater(child)):: Nil case logical.SortPartitions(sortExprs, child) => // Set the requiredDistribution of this SortPartitions to UnspecifiedDistribution. - execution.SortPartitions(sortExprs, planLater(child))() :: Nil + execution.Sort(sortExprs, false, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => diff --git a/src/main/scala/catalyst/plans/physical/partitioning.scala b/src/main/scala/catalyst/plans/physical/partitioning.scala index ee382ede8a79e..dd91b5beb4f95 100644 --- a/src/main/scala/catalyst/plans/physical/partitioning.scala +++ b/src/main/scala/catalyst/plans/physical/partitioning.scala @@ -28,7 +28,6 @@ case object UnspecifiedDistribution extends Distribution */ case object AllTuples extends Distribution -// TODO: Add a BroadcastDistribution? /** * Represents data where tuples that share the same values for the `clustering` * [[catalyst.expressions.Expression Expressions]] will be co-located. Based on the context, this @@ -36,11 +35,11 @@ case object AllTuples extends Distribution * within a single partition. */ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distribution { - if (clustering == Nil) { - throw new IllegalArgumentException("The clustering expressions of a ClusteredDistribution " + - "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + require( + clustering != Nil, + "The clustering expressions of a ClusteredDistribution should not be Nil. " + + "An AllTuples should be used to represent a distribution that only has " + "a single partition.") - } } /** @@ -50,29 +49,15 @@ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distributi * the ordering expressions are contiguous and will never be split across partitions. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { - if (ordering == Nil) { - throw new IllegalArgumentException("The ordering expressions of a OrderedDistribution " + - "should not be Nil. An AllTuples should be used to represent a distribution that only has " + + require( + ordering != Nil, + "The ordering expressions of a OrderedDistribution should not be Nil. " + + "An AllTuples should be used to represent a distribution that only has " + "a single partition.") - } def clustering = ordering.map(_.child).toSet } -object Distribution { - def getSpecifiedDistribution(expressions: Seq[Expression]): Distribution = { - if (expressions == Nil) { - AllTuples - } else { - if (expressions.forall(exp => exp.isInstanceOf[SortOrder])) { - OrderedDistribution(expressions.asInstanceOf[Seq[SortOrder]]) - } else { - ClusteredDistribution(expressions) - } - } - } -} - sealed trait Partitioning { /** Returns the number of partitions that the data is split across */ val numPartitions: Int @@ -97,12 +82,12 @@ sealed trait Partitioning { } case class UnknownPartitioning(numPartitions: Int) extends Partitioning { - def satisfies(required: Distribution): Boolean = required match { + override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true case _ => false } - def compatibleWith(other: Partitioning): Boolean = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case UnknownPartitioning(_) => true case _ => false } @@ -111,9 +96,9 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case object SinglePartition extends Partitioning { val numPartitions = 1 - def satisfies(required: Distribution): Boolean = true + override def satisfies(required: Distribution): Boolean = true - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -122,9 +107,9 @@ case object SinglePartition extends Partitioning { case object BroadcastPartitioning extends Partitioning { val numPartitions = 1 - def satisfies(required: Distribution): Boolean = true + override def satisfies(required: Distribution): Boolean = true - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case SinglePartition => true case _ => false } @@ -146,14 +131,14 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) lazy val clusteringSet = expressions.toSet - def satisfies(required: Distribution): Boolean = required match { + override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true case ClusteredDistribution(requiredClustering) => clusteringSet.subsetOf(requiredClustering.toSet) case _ => false } - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false @@ -180,7 +165,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) lazy val clusteringSet = ordering.map(_.child).toSet - def satisfies(required: Distribution): Boolean = required match { + override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true case OrderedDistribution(requiredOrdering) => val minSize = Seq(requiredOrdering.size, ordering.size).min @@ -190,7 +175,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case _ => false } - def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning) = other match { case BroadcastPartitioning => true case r: RangePartitioning if r == this => true case _ => false diff --git a/src/test/scala/catalyst/execution/DistributionTests.scala b/src/test/scala/catalyst/execution/DistributionTests.scala index 0485ff6ab7bb1..2a3d85b5db40b 100644 --- a/src/test/scala/catalyst/execution/DistributionTests.scala +++ b/src/test/scala/catalyst/execution/DistributionTests.scala @@ -5,7 +5,6 @@ package physical import org.scalatest.FunSuite import catalyst.dsl._ -import catalyst.plans.physical.Distribution._ class DistributionTests extends FunSuite { @@ -34,48 +33,48 @@ class DistributionTests extends FunSuite { checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( SinglePartition, - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) // Cases which need an exchange between two data properties. checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('b, 'c)), + ClusteredDistribution(Seq('b, 'c)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('d, 'e)), + ClusteredDistribution(Seq('d, 'e)), false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Nil), + AllTuples, false) checkSatisfied( HashPartitioning(Seq('a, 'b, 'c), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) checkSatisfied( HashPartitioning(Seq('b, 'c), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), false) // TODO: We should check functional dependencies @@ -96,32 +95,32 @@ class DistributionTests extends FunSuite { checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc)), + OrderedDistribution(Seq('a.asc, 'b.asc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq('a, 'b, 'c)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('c, 'b, 'a)), + ClusteredDistribution(Seq('c, 'b, 'a)), true) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('b, 'c, 'a, 'd)), + ClusteredDistribution(Seq('b, 'c, 'a, 'd)), true) // Cases which need an exchange between two data properties. @@ -131,27 +130,27 @@ class DistributionTests extends FunSuite { // and the parallelism. checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('b.asc, 'a.asc)), + OrderedDistribution(Seq('b.asc, 'a.asc)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('a, 'b)), + ClusteredDistribution(Seq('a, 'b)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Seq('c, 'd)), + ClusteredDistribution(Seq('c, 'd)), false) checkSatisfied( RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - getSpecifiedDistribution(Nil), + AllTuples, false) } } \ No newline at end of file From f47c2f6f3572cb15da916c0efab7839e485ec905 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 31 Jan 2014 14:32:00 +0800 Subject: [PATCH 32/34] set outputPartitioning in BroadcastNestedLoopJoin --- src/main/scala/catalyst/execution/joins.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/scala/catalyst/execution/joins.scala b/src/main/scala/catalyst/execution/joins.scala index fc9e38df7f995..39a172b666f69 100644 --- a/src/main/scala/catalyst/execution/joins.scala +++ b/src/main/scala/catalyst/execution/joins.scala @@ -68,7 +68,9 @@ case class BroadcastNestedLoopJoin( streamed: SharkPlan, broadcast: SharkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SharkContext) extends BinaryNode { - // TODO: Override requiredChildDistribution and outputPartitioning. + // TODO: Override requiredChildDistribution. + + override def outputPartitioning: Partitioning = streamed.outputPartitioning override def otherCopyArgs = sc :: Nil From fc67b5078c23c88b6387cf2b948d84a99cc87e08 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 1 Feb 2014 08:46:18 +0800 Subject: [PATCH 33/34] Check for a Sort operator with the global flag set instead of an Exchange operator with a RangePartitioning. --- src/test/scala/catalyst/execution/HiveComparisonTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/catalyst/execution/HiveComparisonTest.scala b/src/test/scala/catalyst/execution/HiveComparisonTest.scala index 1b4d01c3ce3a9..4a557c06c6daf 100644 --- a/src/test/scala/catalyst/execution/HiveComparisonTest.scala +++ b/src/test/scala/catalyst/execution/HiveComparisonTest.scala @@ -109,7 +109,7 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G case _ => // TODO: Really we only care about the final total ordering here... val isOrdered = sharkQuery.executedPlan.collect { - case s @ Exchange(r: RangePartitioning, _) => s + case s @ Sort(_, global, _) if global => s }.nonEmpty // If the query results aren't sorted, then sort them to ensure deterministic answers. if (!isOrdered) answer.sorted else answer From 45b334b4d06d254c3b9a8f03b2e64f14b48a3c88 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 1 Feb 2014 09:11:07 +0800 Subject: [PATCH 34/34] fix comments --- src/main/scala/catalyst/execution/PlanningStrategies.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/catalyst/execution/PlanningStrategies.scala b/src/main/scala/catalyst/execution/PlanningStrategies.scala index fd9e5e291d98b..10f0f051bddc8 100644 --- a/src/main/scala/catalyst/execution/PlanningStrategies.scala +++ b/src/main/scala/catalyst/execution/PlanningStrategies.scala @@ -143,10 +143,11 @@ trait PlanningStrategies { object BasicOperators extends Strategy { def apply(plan: LogicalPlan): Seq[SharkPlan] = plan match { case logical.Sort(sortExprs, child) => - // Set the requiredDistribution of this SortPartitions to OrderedDistribution. + // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, true, planLater(child)):: Nil case logical.SortPartitions(sortExprs, child) => - // Set the requiredDistribution of this SortPartitions to UnspecifiedDistribution. + // This sort only sort tuples within a partition. Its requiredDistribution will be + // an UnspecifiedDistribution. execution.Sort(sortExprs, false, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil