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
Remove HiveTest singleton
This allows us to use custom SparkContexts in hive tests.
  • Loading branch information
Andrew Or committed Aug 5, 2015
commit d1d1449f3c01c8bb08cc56e9169180313f187bf3
5 changes: 2 additions & 3 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -333,11 +333,12 @@ object SQL {

object Hive {

// TODO: check me, will this work?
lazy val settings = Seq(
javaOptions += "-XX:MaxPermSize=256m",
// Specially disable assertions since some Hive tests fail them
javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"),
// Multiple queries rely on the TestHive singleton. See comments there for more details.
// TODO: re-enable this now that we've gotten rid of the TestHive singleton?
parallelExecution in Test := false,
// Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings
// only for this subproject.
Expand All @@ -356,8 +357,6 @@ object Hive {
|import org.apache.spark.sql.execution
|import org.apache.spark.sql.functions._
|import org.apache.spark.sql.hive._
|import org.apache.spark.sql.hive.test.TestHive._
|import org.apache.spark.sql.hive.test.TestHive.implicits._
|import org.apache.spark.sql.types._""".stripMargin,
cleanupCommands in console := "sparkContext.stop()",
// Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce
Expand Down
1 change: 0 additions & 1 deletion sql/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,6 @@ import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution
import org.apache.spark.sql.functions._
import org.apache.spark.sql.hive._
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.types._
Type in expressions to have them evaluated.
Type :help for more information.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,19 +20,18 @@ package org.apache.spark.sql.hive.execution
import java.io.File

import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.hive.test.TestHive

/**
* Runs the test cases that are included in the hive distribution with hash joins.
*/
class HashJoinCompatibilitySuite extends HiveCompatibilitySuite {
override def beforeAll() {
super.beforeAll()
TestHive.setConf(SQLConf.SORTMERGE_JOIN, false)
ctx.setConf(SQLConf.SORTMERGE_JOIN, false)
}

override def afterAll() {
TestHive.setConf(SQLConf.SORTMERGE_JOIN, true)
ctx.setConf(SQLConf.SORTMERGE_JOIN, true)
super.afterAll()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,41 +23,41 @@ import java.util.{Locale, TimeZone}
import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.hive.test.TestHive

/**
* Runs the test cases that are included in the hive distribution.
*/
class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {

// TODO: bundle in jar files... get from classpath
private lazy val hiveQueryDir = TestHive.getHiveFile(
private lazy val hiveQueryDir = ctx.getHiveFile(
"ql/src/test/queries/clientpositive".split("/").mkString(File.separator))

private val originalTimeZone = TimeZone.getDefault
private val originalLocale = Locale.getDefault
private val originalColumnBatchSize = TestHive.conf.columnBatchSize
private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning
private val originalColumnBatchSize = ctx.conf.columnBatchSize
private val originalInMemoryPartitionPruning = ctx.conf.inMemoryPartitionPruning

def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f)

override def beforeAll() {
TestHive.cacheTables = true
ctx.cacheTables = true
// Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
// Add Locale setting
Locale.setDefault(Locale.US)
// Set a relatively small column batch size for testing purposes
TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, 5)
ctx.setConf(SQLConf.COLUMN_BATCH_SIZE, 5)
// Enable in-memory partition pruning for testing purposes
TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true)
ctx.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true)
}

override def afterAll() {
TestHive.cacheTables = false
ctx.cacheTables = false
TimeZone.setDefault(originalTimeZone)
Locale.setDefault(originalLocale)
TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize)
TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning)
ctx.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize)
ctx.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning)
}

/** A list of tests deemed out of scope currently and thus completely disregarded. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,6 @@ import java.util.{Locale, TimeZone}

import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.util.Utils

/**
Expand All @@ -33,12 +31,14 @@ import org.apache.spark.util.Utils
* files, every `createQueryTest` calls should explicitly set `reset` to `false`.
*/
class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfter {
import ctx._

private val originalTimeZone = TimeZone.getDefault
private val originalLocale = Locale.getDefault
private val testTempDir = Utils.createTempDir()

override def beforeAll() {
TestHive.cacheTables = true
ctx.cacheTables = true
// Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
// Add Locale setting
Expand All @@ -59,7 +59,7 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte
| p_retailprice DOUBLE,
| p_comment STRING)
""".stripMargin)
val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").getCanonicalPath
val testData1 = ctx.getHiveFile("data/files/part_tiny.txt").getCanonicalPath
sql(
s"""
|LOAD DATA LOCAL INPATH '$testData1' overwrite into table part
Expand All @@ -83,7 +83,7 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte
|row format delimited
|fields terminated by '|'
""".stripMargin)
val testData2 = TestHive.getHiveFile("data/files/over1k").getCanonicalPath
val testData2 = ctx.getHiveFile("data/files/over1k").getCanonicalPath
sql(
s"""
|LOAD DATA LOCAL INPATH '$testData2' overwrite into table over1k
Expand All @@ -100,10 +100,10 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte
}

override def afterAll() {
TestHive.cacheTables = false
ctx.cacheTables = false
TimeZone.setDefault(originalTimeZone)
Locale.setDefault(originalLocale)
TestHive.reset()
ctx.reset()
}

/////////////////////////////////////////////////////////////////////////////
Expand Down Expand Up @@ -766,7 +766,7 @@ class HiveWindowFunctionQueryFileSuite
private val testTempDir = Utils.createTempDir()

override def beforeAll() {
TestHive.cacheTables = true
ctx.cacheTables = true
// Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*)
TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
// Add Locale setting
Expand All @@ -783,10 +783,10 @@ class HiveWindowFunctionQueryFileSuite
}

override def afterAll() {
TestHive.cacheTables = false
ctx.cacheTables = false
TimeZone.setDefault(originalTimeZone)
Locale.setDefault(originalLocale)
TestHive.reset()
ctx.reset()
}

override def blackList: Seq[String] = Seq(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,30 +42,12 @@ import org.apache.spark.{SparkConf, SparkContext}
/* Implicit conversions */
import scala.collection.JavaConversions._

// TODO: remove it
object TestHive
extends TestHiveContext(
new SparkContext(
System.getProperty("spark.sql.test.master", "local[32]"),
"TestSQLContext",
new SparkConf()
.set("spark.sql.test", "")
.set("spark.sql.hive.metastore.barrierPrefixes",
"org.apache.spark.sql.hive.execution.PairSerDe")
.set("spark.buffer.pageSize", "4m")
// SPARK-8910
.set("spark.ui.enabled", "false")))

/**
* A locally running test instance of Spark's Hive execution engine.
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 deprecate this instead of removing it? I think its fairly common for people to use this for their own unit tests.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

do you mean outside of Spark? The point is to never use this in Spark tests, since even referencing it will destabilize the tests because it leaves two overlapping SparkContexts in the same JVM

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, we can avoid it in our own tests, but I know for a fact that it is used externally.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

OK, I've added these back with a deprecation warning in the latest commit.

*
* Data from [[testTables]] will be automatically loaded whenever a query is run over those tables.
* Calling [[reset]] will delete all tables and other state in the database, leaving the database
* in a "clean" state.
*
* TestHive is singleton object version of this class because instantiating multiple copies of the
* hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of
* test cases that rely on TestHive must be serialized.
*/
class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
self =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import org.apache.spark.sql.*;
import org.apache.spark.sql.expressions.Window;
import org.apache.spark.sql.hive.HiveContext;
import org.apache.spark.sql.hive.test.TestHive$;
import org.apache.spark.sql.hive.test.TestHiveContext;

public class JavaDataFrameSuite {
private transient JavaSparkContext sc;
Expand All @@ -47,7 +47,7 @@ private void checkAnswer(DataFrame actual, List<Row> expected) {

@Before
public void setUp() throws IOException {
hc = TestHive$.MODULE$;
hc = new TestHiveContext();
sc = new JavaSparkContext(hc.sparkContext());

List<String> jsonObjects = new ArrayList<String>(10);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@
import org.apache.spark.sql.QueryTest$;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.hive.HiveContext;
import org.apache.spark.sql.hive.test.TestHive$;
import org.apache.spark.sql.hive.test.TestHiveContext;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
Expand All @@ -63,7 +63,7 @@ private void checkAnswer(DataFrame actual, List<Row> expected) {

@Before
public void setUp() throws IOException {
sqlContext = TestHive$.MODULE$;
sqlContext = new TestHiveContext();
sc = new JavaSparkContext(sqlContext.sparkContext());

originalDefaultSource = sqlContext.conf().defaultDataSourceName();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,15 @@ package org.apache.spark.sql.hive

import java.io.File

import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation}
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.{SaveMode, AnalysisException, DataFrame, QueryTest}
import org.apache.spark.sql.{SaveMode, AnalysisException, QueryTest}
import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
import org.apache.spark.sql.hive.test.MyTestHiveContext
import org.apache.spark.storage.RDDBlockId
import org.apache.spark.util.Utils

class CachedTableSuite extends QueryTest {
class CachedTableSuite extends QueryTest with MyTestHiveContext {
private val ctx = hiveContext
import ctx._

def rddIdOf(tableName: String): Int = {
val executedPlan = table(tableName).queryExecution.executedPlan
Expand Down Expand Up @@ -95,18 +96,18 @@ class CachedTableSuite extends QueryTest {

test("correct error on uncache of non-cached table") {
intercept[IllegalArgumentException] {
TestHive.uncacheTable("src")
uncacheTable("src")
}
}

test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") {
TestHive.sql("CACHE TABLE src")
sql("CACHE TABLE src")
assertCached(table("src"))
assert(TestHive.isCached("src"), "Table 'src' should be cached")
assert(isCached("src"), "Table 'src' should be cached")

TestHive.sql("UNCACHE TABLE src")
sql("UNCACHE TABLE src")
assertCached(table("src"), 0)
assert(!TestHive.isCached("src"), "Table 'src' should not be cached")
assert(!isCached("src"), "Table 'src' should not be cached")
}

test("CACHE TABLE tableName AS SELECT * FROM anotherTable") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,14 @@ import scala.util.Try
import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.catalyst.util.quietly
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.hive.test.MyTestHiveContext
import org.apache.spark.sql.{AnalysisException, QueryTest}


class ErrorPositionSuite extends QueryTest with BeforeAndAfter {
class ErrorPositionSuite extends QueryTest with BeforeAndAfter with MyTestHiveContext {
private val ctx = hiveContext
import ctx.implicits._
import ctx._

before {
Seq((1, 1, 1)).toDF("a", "a", "b").registerTempTable("dupAttributes")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,24 +19,25 @@ package org.apache.spark.sql.hive

import org.apache.spark.sql.{DataFrame, QueryTest}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.scalatest.BeforeAndAfterAll
import org.apache.spark.sql.hive.test.MyTestHiveContext

// TODO ideally we should put the test suite into the package `sql`, as
// `hive` package is optional in compiling, however, `SQLContext.sql` doesn't
// support the `cube` or `rollup` yet.
class HiveDataFrameAnalyticsSuite extends QueryTest with BeforeAndAfterAll {
class HiveDataFrameAnalyticsSuite extends QueryTest with MyTestHiveContext {
private val ctx = hiveContext
import ctx.implicits._
import ctx._

private var testData: DataFrame = _

override def beforeAll() {
testData = Seq((1, 2), (2, 4)).toDF("a", "b")
TestHive.registerDataFrameAsTable(testData, "mytable")
registerDataFrameAsTable(testData, "mytable")
}

override def afterAll(): Unit = {
TestHive.dropTempTable("mytable")
dropTempTable("mytable")
}

test("rollup") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,12 @@
package org.apache.spark.sql.hive

import org.apache.spark.sql.{Row, QueryTest}
import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.hive.test.MyTestHiveContext


class HiveDataFrameJoinSuite extends QueryTest {
class HiveDataFrameJoinSuite extends QueryTest with MyTestHiveContext {
private val ctx = hiveContext
import ctx.implicits._

// We should move this into SQL package if we make case sensitivity configurable in SQL.
test("join - self join auto resolve ambiguity with case insensitivity") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@ package org.apache.spark.sql.hive
import org.apache.spark.sql.{Row, QueryTest}
import org.apache.spark.sql.expressions.Window
import org.apache.spark.sql.functions._
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.hive.test.MyTestHiveContext

class HiveDataFrameWindowSuite extends QueryTest {
class HiveDataFrameWindowSuite extends QueryTest with MyTestHiveContext {
private val ctx = hiveContext
import ctx.implicits._
import ctx._

test("reuse window partitionBy") {
val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value")
Expand Down
Loading