Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
WIP
  • Loading branch information
JoshRosen committed Jun 17, 2015
commit a46144a4e3d10c4b2d872250518fe06f659eb90f
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution

import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder}

class SortSuite extends SparkPlanTest {

test("basic sorting using ExternalSort") {

val input = Seq(
("Hello", 4),
("Hello", 1),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we have more data types here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure; this was intended to be more of a demonstration of SparkPlanTest than an extensive test for sort. I didn't want to create a class named SparkPlanTestSuite since that seemed likely to confuse people since it sounds like the name of a test suite base class.

("World", 8)
)

val sortOrder = Seq(
SortOrder('_1, Ascending),
SortOrder('_2, Ascending)
)

checkAnswer(
input,
child => new ExternalSort(sortOrder, global = false, child),
input.sorted
)

}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution

import scala.util.control.NonFatal
import scala.reflect.runtime.universe.TypeTag

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.test.TestSQLContext
import org.apache.spark.sql.{Row, DataFrame}
import org.apache.spark.sql.catalyst.util._

/**
* Base class for writing tests for individual physical operators. For an example of how this class
* can be used, see [[SortSuite]].
*/
class SparkPlanTest extends SparkFunSuite {

/**
* Runs the plan and makes sure the answer matches the expected result.
* @param input the input data to be used.
* @param planFunction a function which accepts the input SparkPlan and uses it to instantiate the
* physical operator that's being tested.
* @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s.
*/
protected def checkAnswer(
input: DataFrame,
planFunction: SparkPlan => SparkPlan,
expectedAnswer: Seq[Row]): Unit = {
SparkPlanTest.checkAnswer(input, planFunction, expectedAnswer) match {
case Some(errorMessage) => fail(errorMessage)
case None =>
}
}

/**
* Runs the plan and makes sure the answer matches the expected result.
* @param input the input data to be used.
* @param planFunction a function which accepts the input SparkPlan and uses it to instantiate the
* physical operator that's being tested.
* @param expectedAnswer the expected result in a [[Seq]] of [[Product]]s.
*/
protected def checkAnswer[A <: Product : TypeTag](
input: Seq[A],
planFunction: SparkPlan => SparkPlan,
expectedAnswer: Seq[A]): Unit = {
val inputDf = TestSQLContext.createDataFrame(input)
val expectedRows = expectedAnswer.map(t => Row.apply(t))
SparkPlanTest.checkAnswer(inputDf, planFunction, expectedRows) match {
case Some(errorMessage) => fail(errorMessage)
case None =>
}
}
}

/**
* Helper methods for writing tests of individual physical operators.
*/
object SparkPlanTest {

/**
* Runs the plan and makes sure the answer matches the expected result.
* @param input the input data to be used.
* @param planFunction a function which accepts the input SparkPlan and uses it to instantiate the
* physical operator that's being tested.
* @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s.
*/
def checkAnswer(
input: DataFrame,
planFunction: SparkPlan => SparkPlan,
expectedAnswer: Seq[Row]): Option[String] = {

val outputPlan = planFunction(input.queryExecution.sparkPlan)

def prepareAnswer(answer: Seq[Row]): Seq[Row] = {
// Converts data to types that we can do equality comparison using Scala collections.
// For BigDecimal type, the Scala type has a better definition of equality test (similar to
// Java's java.math.BigDecimal.compareTo).
// For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for
// equality test.
// This function is copied from Catalyst's QueryTest
val converted: Seq[Row] = answer.map { s =>
Row.fromSeq(s.toSeq.map {
case d: java.math.BigDecimal => BigDecimal(d)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the problem with java.math.BigDecimal?

scala> val b = new java.math.BigDecimal(10)
b: java.math.BigDecimal = 10

scala> val c = new java.math.BigDecimal(10)
c: java.math.BigDecimal = 10

scala> b == c
res6: Boolean = true

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure; as the comment for converted notes, this is duplicated from Catalyst's QueryTest.

case b: Array[Byte] => b.toSeq
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The equality check of Array[Byte] will be fixed by #6876

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, gotcha. I'd be happy to block on this. One consideration, though: we might want to backport this test helper to some of our maintenance branches so that we don't get build failures when backporting regression tests which use SparkPlanTest. In that case, we might also need to backport those other byte comparison fixes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of blocking lets just make a note in the JIRA to remove these hacks if possible later. I'd like to get this in today and I agree with you backporting concerns.

case o => o
})
}
converted.sortBy(_.toString())
}

val sparkAnswer: Seq[Row] = try {
outputPlan.executeCollect().toSeq
} catch {
case NonFatal(e) =>
val errorMessage =
s"""
| Exception thrown while executing Spark plan:
| $outputPlan
| == Exception ==
| $e
| ${org.apache.spark.sql.catalyst.util.stackTraceToString(e)}
""".stripMargin
return Some(errorMessage)
}

if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we use ``!==` here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think so, since we want to throw our own custom error message rather than letting ScalaTest do it.

val errorMessage =
s"""
| Results do not match for Spark plan:
| $outputPlan
| == Results ==
| ${sideBySide(
s"== Correct Answer - ${expectedAnswer.size} ==" +:
prepareAnswer(expectedAnswer).map(_.toString()),
s"== Spark Answer - ${sparkAnswer.size} ==" +:
prepareAnswer(sparkAnswer).map(_.toString())).mkString("\n")}
""".stripMargin
return Some(errorMessage)
}

None
}
}