Skip to content

Commit

Permalink
Add even more tests.
Browse files Browse the repository at this point in the history
  • Loading branch information
JoshRosen committed Aug 6, 2015
1 parent 4f08278 commit 8dbc845
Showing 1 changed file with 54 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -209,6 +209,10 @@ class PlannerSuite extends SparkFunSuite with SQLTestUtils {

// --- Unit tests of EnsureRequirements ---------------------------------------------------------

// When it comes to testing whether EnsureRequirements properly ensures distribution requirements,
// there two dimensions that need to be considered: are the child partitionings compatible and
// do they satisfy the distribution requirements? As a result, we need at least four test cases.

private def assertDistributionRequirementsAreSatisfied(outputPlan: SparkPlan): Unit = {
if (outputPlan.requiresChildPartitioningsToBeCompatible) {
val childPartitionings = outputPlan.children.map(_.outputPartitioning)
Expand All @@ -223,7 +227,7 @@ class PlannerSuite extends SparkFunSuite with SQLTestUtils {
}
}

test("EnsureRequirements ensures that child partitionings guarantee each other, if required") {
test("EnsureRequirements with incompatible child partitionings which satisfy distribution") {
// Consider an operator that requires inputs that are clustered by two expressions (e.g.
// sort merge join where there are multiple columns in the equi-join condition)
val clusteringA = Literal(1) :: Nil
Expand All @@ -241,8 +245,8 @@ class PlannerSuite extends SparkFunSuite with SQLTestUtils {
assert(!rightPartitioning.guarantees(leftPartitioning))
val inputPlan = DummyPlan(
children = Seq(
DummyPlan(outputPartitioning = HashPartitioning(clusteringA, 1)),
DummyPlan(outputPartitioning = HashPartitioning(clusteringB, 1))
DummyPlan(outputPartitioning = leftPartitioning),
DummyPlan(outputPartitioning = rightPartitioning)
),
requiresChildPartitioningsToBeCompatible = true,
requiredChildDistribution = Seq(distribution, distribution),
Expand All @@ -251,14 +255,13 @@ class PlannerSuite extends SparkFunSuite with SQLTestUtils {
val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan)
assertDistributionRequirementsAreSatisfied(outputPlan)
if (outputPlan.collect { case Exchange(_, _) => true }.isEmpty) {
fail(s"Exchanges should have been added:\n$outputPlan")
fail(s"Exchange should have been added:\n$outputPlan")
}
}

test("EnsureRequirements ensures that children produce same number of partitions, if required") {
test("EnsureRequirements with child partitionings with different numbers of output partitions") {
// This is similar to the previous test, except it checks that partitionings are not compatible
// unless they produce the same number of partitions. This requirement is also enforced via
// assertions in Exchange.
// unless they produce the same number of partitions.
val clustering = Literal(1) :: Nil
val distribution = ClusteredDistribution(clustering)
val inputPlan = DummyPlan(
Expand All @@ -274,6 +277,50 @@ class PlannerSuite extends SparkFunSuite with SQLTestUtils {
assertDistributionRequirementsAreSatisfied(outputPlan)
}

test("EnsureRequirements with compatible child partitionings that do not satisfy distribution") {
val distribution = ClusteredDistribution(Literal(1) :: Nil)
// The left and right inputs have compatible partitionings but they do not satisfy the
// distribution because they are clustered on different columns. Thus, we need to shuffle.
val childPartitioning = HashPartitioning(Literal(2) :: Nil, 1)
assert(!childPartitioning.satisfies(distribution))
val inputPlan = DummyPlan(
children = Seq(
DummyPlan(outputPartitioning = childPartitioning),
DummyPlan(outputPartitioning = childPartitioning)
),
requiresChildPartitioningsToBeCompatible = true,
requiredChildDistribution = Seq(distribution, distribution),
requiredChildOrdering = Seq(Seq.empty, Seq.empty)
)
val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan)
assertDistributionRequirementsAreSatisfied(outputPlan)
if (outputPlan.collect { case Exchange(_, _) => true }.isEmpty) {
fail(s"Exchange should have been added:\n$outputPlan")
}
}

test("EnsureRequirements with compatible child partitionings that satisfy distribution") {
// In this case, all requirements are satisfied and no exchange should be added.
val distribution = ClusteredDistribution(Literal(1) :: Nil)
val childPartitioning = HashPartitioning(Literal(1) :: Nil, 5)
assert(childPartitioning.satisfies(distribution))
val inputPlan = DummyPlan(
children = Seq(
DummyPlan(outputPartitioning = childPartitioning),
DummyPlan(outputPartitioning = childPartitioning)
),
requiresChildPartitioningsToBeCompatible = true,
requiredChildDistribution = Seq(distribution, distribution),
requiredChildOrdering = Seq(Seq.empty, Seq.empty)
)
val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan)
assertDistributionRequirementsAreSatisfied(outputPlan)
if (outputPlan.collect { case Exchange(_, _) => true }.nonEmpty) {
fail(s"Exchange should not have been added:\n$outputPlan")
}
}

// This is a regression test for SPARK-9703
test("EnsureRequirements should not repartition if only ordering requirement is unsatisfied") {
// Consider an operator that imposes both output distribution and ordering requirements on its
// children, such as sort sort merge join. If the distribution requirements are satisfied but
Expand Down

0 comments on commit 8dbc845

Please sign in to comment.