Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
58 commits
Select commit Hold shift + click to select a range
16ca7ac
It compiles!!
Aug 4, 2015
5e6a20c
Refactor SQLTestUtils to reduce duplication
Aug 5, 2015
d1d1449
Remove HiveTest singleton
Aug 5, 2015
d4aafb1
Avoid the need to switch to HiveContexts
Aug 5, 2015
6345cee
Clean up JsonSuite
Aug 5, 2015
68ac6fe
Rename the test traits properly
Aug 5, 2015
b15fdc6
Stop SparkContext in Java SQL tests
Aug 5, 2015
0d74a72
Load test data early in case tables are accessed by name
Aug 5, 2015
eee415d
Refactor implicits into SQLTestUtils
Aug 10, 2015
55d0b1b
Fix Java not serializable exception in tests
Aug 10, 2015
4f59bee
Fix DataSourceTest et al.
Aug 10, 2015
88d4f16
Fix hive tests to use the same pattern
Aug 11, 2015
5fe4bfb
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
c4a22bc
Fix compile after resolving merge conflicts
Aug 11, 2015
d9a8390
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
f5619f8
Fix test compile after resolving merge conflicts
Aug 11, 2015
9395cfa
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
c51b3d8
Fix OuterJoinSuite
Aug 11, 2015
997715e
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
1400770
Fix style
Aug 11, 2015
c92a3b0
Fix MiMa
Aug 11, 2015
4debedf
Clean up inheritance in test util traits
Aug 11, 2015
ca87dda
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
1cf53ad
Create new context in SBT console by default
Aug 11, 2015
bc5c999
Fix SemiJoinSuite
Aug 12, 2015
19fd6c3
Fix InnerJoinSuite
Aug 12, 2015
e1e513e
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
1e4c321
Address comments
Aug 12, 2015
54848e2
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
94f9c77
Revert the removal of some BeforeAndAfters
Aug 12, 2015
bec7d28
Fix a hive test + minor format updates
Aug 12, 2015
24c086d
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
561eacc
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
aaba277
Fix places where we override before / after alls
Aug 12, 2015
aac8f7f
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
40959bb
Fix test
Aug 12, 2015
55d6624
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
4ed58c8
Revert a few merge-conflict-induced unintentional changes
Aug 12, 2015
0ce5638
Minor updates
Aug 13, 2015
814df2f
Add back singletons but deprecate them
Aug 13, 2015
099c74f
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
451fa37
Use consistent name for added back singletons
Aug 13, 2015
9ea7f7c
Fix style
Aug 13, 2015
ddc4b05
Revert "Use consistent name for added back singletons"
Aug 13, 2015
ece3a81
Remove TestSQLContext, but keep TestHive
Aug 13, 2015
94cc3b5
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
8d69bf8
Fix test
Aug 13, 2015
7a0b1ef
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
b58ae73
Fix tests
Aug 13, 2015
828144f
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
c4d44c9
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
48af8e4
Fix another before / after alls
Aug 13, 2015
0606c82
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
f599bbc
Revert all Hive related changes
Aug 13, 2015
aed7fc7
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
d85a6d8
Add a shorthand for sqlContext.sql / ctx.sql
Aug 13, 2015
0b60325
Fix hive test compile
Aug 13, 2015
821ea67
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
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
Prev Previous commit
Next Next commit
Avoid the need to switch to HiveContexts
This is a clean up to refactor helper test traits and abstract
classes in such a way that is accessible to hive tests.
  • Loading branch information
Andrew Or committed Aug 5, 2015
commit d4aafb16c4201fc31cc0875906f751eee50a9dc1
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import org.apache.spark.sql.test.MyTestSQLContext
private case class BigData(s: String)

class CachedTableSuite extends QueryTest with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._
import org.apache.spark.sql.test.SQLTestUtils

class ColumnExpressionSuite extends QueryTest with SQLTestUtils {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.spark.sql.types.DecimalType


class DataFrameAggregateSuite extends QueryTest with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._
* Test suite for functions in [[org.apache.spark.sql.functions]].
*/
class DataFrameFunctionsSuite extends QueryTest with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.MyTestSQLContext

class DataFrameJoinSuite extends QueryTest with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._
import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint, SQLTestUtils}

class DataFrameSuite extends QueryTest with SQLTestUtils {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.test.MyTestSQLContext


class JoinSuite extends QueryTest with BeforeAndAfterEach with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import org.apache.spark.sql.types._
class MyDialect extends DefaultParserDialect

class SQLQuerySuite extends QueryTest with SQLTestUtils {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.spark.sql.test.SQLTestUtils
private case class FunctionResult(f1: String, f2: String)

class UDFSuite extends QueryTest with SQLTestUtils {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.storage.StorageLevel.MEMORY_ONLY

class InMemoryColumnarQuerySuite extends QueryTest with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.test.MyTestSQLContext

class PartitionBatchPruningSuite extends SparkFunSuite with BeforeAndAfter with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import org.apache.spark.sql.{execution, Row, SQLConf}


class PlannerSuite extends SparkFunSuite with SQLTestUtils {
private val ctx = sqlContextWithData
private val ctx = sqlContext
import ctx.implicits._
import ctx.planner._
import ctx._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,18 +27,25 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.test.MyTestSQLContext


/**
* Base class for writing tests for individual physical operators. For an example of how this
* class's test helper methods can be used, see [[SortSuite]].
*/
abstract class SparkPlanTest extends SparkFunSuite with MyTestSQLContext {
private[sql] abstract class SparkPlanTest extends AbstractSparkPlanTest with MyTestSQLContext {
protected override def _sqlContext: SQLContext = sqlContext
}

/**
* Helper class for testing individual physical operators with a pluggable [[SQLContext]].
*/
private[sql] abstract class AbstractSparkPlanTest extends SparkFunSuite {
protected def _sqlContext: SQLContext

/**
* Creates a DataFrame from a local Seq of Product.
*/
implicit def localSeqToDataFrameHolder[A <: Product : TypeTag](data: Seq[A]): DataFrameHolder = {
sqlContext.implicits.localSeqToDataFrameHolder(data)
_sqlContext.implicits.localSeqToDataFrameHolder(data)
}

/**
Expand Down Expand Up @@ -99,7 +106,7 @@ abstract class SparkPlanTest extends SparkFunSuite with MyTestSQLContext {
planFunction: Seq[SparkPlan] => SparkPlan,
expectedAnswer: Seq[Row],
sortAnswers: Boolean = true): Unit = {
SparkPlanTest.checkAnswer(input, planFunction, expectedAnswer, sortAnswers, sqlContext) match {
SparkPlanTest.checkAnswer(input, planFunction, expectedAnswer, sortAnswers, _sqlContext) match {
case Some(errorMessage) => fail(errorMessage)
case None =>
}
Expand All @@ -123,7 +130,7 @@ abstract class SparkPlanTest extends SparkFunSuite with MyTestSQLContext {
expectedPlanFunction: SparkPlan => SparkPlan,
sortAnswers: Boolean = true): Unit = {
SparkPlanTest.checkAnswer(
input, planFunction, expectedPlanFunction, sortAnswers, sqlContext) match {
input, planFunction, expectedPlanFunction, sortAnswers, _sqlContext) match {
case Some(errorMessage) => fail(errorMessage)
case None =>
}
Expand All @@ -150,13 +157,13 @@ object SparkPlanTest {
planFunction: SparkPlan => SparkPlan,
expectedPlanFunction: SparkPlan => SparkPlan,
sortAnswers: Boolean,
sqlContext: SQLContext): Option[String] = {
_sqlContext: SQLContext): Option[String] = {

val outputPlan = planFunction(input.queryExecution.sparkPlan)
val expectedOutputPlan = expectedPlanFunction(input.queryExecution.sparkPlan)

val expectedAnswer: Seq[Row] = try {
executePlan(expectedOutputPlan, sqlContext)
executePlan(expectedOutputPlan, _sqlContext)
} catch {
case NonFatal(e) =>
val errorMessage =
Expand All @@ -171,7 +178,7 @@ object SparkPlanTest {
}

val actualAnswer: Seq[Row] = try {
executePlan(outputPlan, sqlContext)
executePlan(outputPlan, _sqlContext)
} catch {
case NonFatal(e) =>
val errorMessage =
Expand Down Expand Up @@ -211,12 +218,12 @@ object SparkPlanTest {
planFunction: Seq[SparkPlan] => SparkPlan,
expectedAnswer: Seq[Row],
sortAnswers: Boolean,
sqlContext: SQLContext): Option[String] = {
_sqlContext: SQLContext): Option[String] = {

val outputPlan = planFunction(input.map(_.queryExecution.sparkPlan))

val sparkAnswer: Seq[Row] = try {
executePlan(outputPlan, sqlContext)
executePlan(outputPlan, _sqlContext)
} catch {
case NonFatal(e) =>
val errorMessage =
Expand Down Expand Up @@ -279,10 +286,10 @@ object SparkPlanTest {
}
}

private def executePlan(outputPlan: SparkPlan, sqlContext: SQLContext): Seq[Row] = {
private def executePlan(outputPlan: SparkPlan, _sqlContext: SQLContext): Seq[Row] = {
// A very simple resolver to make writing tests easier. In contrast to the real resolver
// this is always case sensitive and does not try to handle scoping or complex type resolution.
val resolvedPlan = sqlContext.prepareForExecution.execute(
val resolvedPlan = _sqlContext.prepareForExecution.execute(
outputPlan transform {
case plan: SparkPlan =>
val inputMap = plan.children.flatMap(_.output).map(a => (a.name, a)).toMap
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.test.MyTestSQLContext

class DebuggingSuite extends SparkFunSuite with MyTestSQLContext {
private val ctx = sqlContextWithData
private val ctx = sqlContext

test("DataFrame.debug()") {
ctx.testData.debug()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import org.apache.spark.sql.types._
import org.apache.spark.util.Utils

class JsonSuite extends QueryTest with TestJsonData with MyTestSQLContext {
private val _ctx = sqlContextWithData
private val _ctx = sqlContext
import _ctx.implicits._
import _ctx._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,18 +16,35 @@
*/

package org.apache.spark.sql.parquet

import java.io.File

import scala.collection.JavaConversions._

import org.scalatest.BeforeAndAfterAll
import org.apache.hadoop.fs.Path
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.schema.MessageType

import org.apache.spark.sql.QueryTest
import org.apache.spark.util.Utils

abstract class ParquetCompatibilityTest extends QueryTest with ParquetTest {
/**
* Helper class for testing Parquet compatibility.
*/
private[sql] abstract class ParquetCompatibilityTest
extends AbstractParquetCompatibilityTest
with ParquetTest

/**
* Abstract helper class for testing Parquet compatibility with a pluggable
* [[org.apache.spark.sql.SQLContext]].
*/
private[sql] abstract class AbstractParquetCompatibilityTest
extends QueryTest
with AbstractParquetTest
with BeforeAndAfterAll {

protected var parquetStore: File = _

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@ import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.{DataFrame, SaveMode}
import org.apache.spark.sql.test.{AbstractSQLTestUtils, SQLTestUtils}
import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}

/**
* A helper trait that provides convenient facilities for Parquet testing.
Expand All @@ -33,7 +33,13 @@ import org.apache.spark.sql.{DataFrame, SaveMode}
* convenient to use tuples rather than special case classes when writing test cases/suites.
* Especially, `Tuple1.apply` can be used to easily wrap a single type/value.
*/
private[sql] trait ParquetTest extends SparkFunSuite with SQLTestUtils {
private[sql] trait ParquetTest extends AbstractParquetTest with SQLTestUtils

/**
* Abstract helper trait for Parquet tests with a pluggable [[SQLContext]].
*/
private[sql] trait AbstractParquetTest extends SparkFunSuite with AbstractSQLTestUtils {
protected def _sqlContext: SQLContext

/**
* Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f`
Expand All @@ -43,7 +49,7 @@ private[sql] trait ParquetTest extends SparkFunSuite with SQLTestUtils {
(data: Seq[T])
(f: String => Unit): Unit = {
withTempPath { file =>
sqlContext.createDataFrame(data).write.parquet(file.getCanonicalPath)
_sqlContext.createDataFrame(data).write.parquet(file.getCanonicalPath)
f(file.getCanonicalPath)
}
}
Expand All @@ -55,7 +61,7 @@ private[sql] trait ParquetTest extends SparkFunSuite with SQLTestUtils {
protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag]
(data: Seq[T])
(f: DataFrame => Unit): Unit = {
withParquetFile(data)(path => f(sqlContext.read.parquet(path)))
withParquetFile(data)(path => f(_sqlContext.read.parquet(path)))
}

/**
Expand All @@ -67,14 +73,14 @@ private[sql] trait ParquetTest extends SparkFunSuite with SQLTestUtils {
(data: Seq[T], tableName: String)
(f: => Unit): Unit = {
withParquetDataFrame(data) { df =>
sqlContext.registerDataFrameAsTable(df, tableName)
_sqlContext.registerDataFrameAsTable(df, tableName)
withTempTable(tableName)(f)
}
}

protected def makeParquetFile[T <: Product: ClassTag: TypeTag](
data: Seq[T], path: File): Unit = {
sqlContext.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath)
_sqlContext.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath)
}

protected def makeParquetFile[T <: Product: ClassTag: TypeTag](
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,45 +64,36 @@ private[spark] class MyLocalSQLContext(sc: SparkContext) extends SQLContext(sc)
private[spark] trait MyTestSQLContext extends SparkFunSuite with BeforeAndAfterAll {

/**
* The [[SQLContext]] to use for all tests in this suite.
* The [[MyLocalSQLContext]] to use for all tests in this suite.
*
* By default, the underlying [[SparkContext]] will be run in local mode with the default
* test configurations.
*/
private var _ctx: SQLContext = new MyLocalSQLContext

/** The [[SQLContext]] to use for all tests in this suite. */
protected def sqlContext: SQLContext = _ctx
private var _ctx: MyLocalSQLContext = new MyLocalSQLContext

/**
* The [[MyLocalSQLContext]] to use for all tests in this suite.
* This one comes with all the data prepared in advance.
*/
protected def sqlContextWithData: MyLocalSQLContext = {
_ctx match {
case local: MyLocalSQLContext => local
case _ => fail("this SQLContext does not have data prepared in advance")
}
}
protected def sqlContext: MyLocalSQLContext = _ctx

/**
* Switch to the provided [[SQLContext]].
* Switch to the provided [[MyLocalSQLContext]].
*
* This stops the underlying [[SparkContext]] and expects a new one to be created.
* This is needed because only one [[SparkContext]] is allowed per JVM.
*/
protected def switchSQLContext(newContext: () => SQLContext): Unit = {
protected def switchSQLContext(newContext: () => MyLocalSQLContext): Unit = {
if (_ctx != null) {
_ctx.sparkContext.stop()
_ctx = newContext()
}
}

/**
* Execute the given block of code with a custom [[SQLContext]].
* At the end of the method, a [[MyLocalSQLContext]] will be restored.
* Execute the given block of code with a custom [[MyLocalSQLContext]].
* At the end of the method, the default [[MyLocalSQLContext]] will be restored.
*/
protected def withSQLContext[T](newContext: () => SQLContext)(body: => T) {
protected def withSQLContext[T](newContext: () => MyLocalSQLContext)(body: => T) {
switchSQLContext(newContext)
try {
body
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,9 @@ import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.SQLContext
import org.apache.spark.util.Utils

/**
* General helper trait for common functionality in SQL tests.
*/
private[spark] trait SQLTestUtils
extends SparkFunSuite
with AbstractSQLTestUtils
Expand All @@ -34,6 +37,9 @@ private[spark] trait SQLTestUtils
protected final override def _sqlContext = sqlContext
}

/**
* Abstract helper trait for SQL tests with a pluggable [[SQLContext]].
*/
private[spark] trait AbstractSQLTestUtils { this: SparkFunSuite =>
protected def _sqlContext: SQLContext

Expand Down
Loading