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
Fix compile after resolving merge conflicts
  • Loading branch information
Andrew Or committed Aug 11, 2015
commit c4a22bcb8d26362f741c9b24f31552694b3d4f64
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@ package org.apache.spark.sql

import java.util.Random

import org.scalatest.Matchers._

import org.apache.spark.sql.functions.col
import org.apache.spark.sql.test.SQLTestUtils

Expand All @@ -31,7 +29,7 @@ class DataFrameStatSuite extends QueryTest with SQLTestUtils {

test("sample with replacement") {
val n = 100
val data = sqlCtx.sparkContext.parallelize(1 to n, 2).toDF("id")
val data = ctx.sparkContext.parallelize(1 to n, 2).toDF("id")
checkAnswer(
data.sample(withReplacement = true, 0.05, seed = 13),
Seq(5, 10, 52, 73).map(Row(_))
Expand All @@ -40,7 +38,7 @@ class DataFrameStatSuite extends QueryTest with SQLTestUtils {

test("sample without replacement") {
val n = 100
val data = sqlCtx.sparkContext.parallelize(1 to n, 2).toDF("id")
val data = ctx.sparkContext.parallelize(1 to n, 2).toDF("id")
checkAnswer(
data.sample(withReplacement = false, 0.05, seed = 13),
Seq(16, 23, 88, 100).map(Row(_))
Expand All @@ -49,7 +47,7 @@ class DataFrameStatSuite extends QueryTest with SQLTestUtils {

test("randomSplit") {
val n = 600
val data = sqlCtx.sparkContext.parallelize(1 to n, 2).toDF("id")
val data = ctx.sparkContext.parallelize(1 to n, 2).toDF("id")
for (seed <- 1 to 5) {
val splits = data.randomSplit(Array[Double](1, 2, 3), seed)
assert(splits.length == 3, "wrong number of splits")
Expand Down Expand Up @@ -166,7 +164,7 @@ class DataFrameStatSuite extends QueryTest with SQLTestUtils {
}

test("Frequent Items 2") {
val rows = sqlCtx.sparkContext.parallelize(Seq.empty[Int], 4)
val rows = ctx.sparkContext.parallelize(Seq.empty[Int], 4)
// this is a regression test, where when merging partitions, we omitted values with higher
// counts than those that existed in the map when the map was full. This test should also fail
// if anything like SPARK-9614 is observed once again
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@

package org.apache.spark.sql

class JsonFunctionsSuite extends QueryTest {
import org.apache.spark.sql.test.SQLTestUtils

private lazy val ctx = org.apache.spark.sql.test.TestSQLContext
import ctx.implicits._
class JsonFunctionsSuite extends QueryTest with SQLTestUtils {
import testImplicits._

test("function get_json_object") {
val df: DataFrame = Seq(("""{"name": "alice", "age": 5}""", "")).toDF("a", "b")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,8 @@ class PlannerSuite extends SparkFunSuite with SQLTestUtils {
setupTestData()

private def testPartialAggregationPlan(query: LogicalPlan): Unit = {
val _ctx = ctx
import _ctx.planner._
val plannedOption = HashAggregation(query).headOption.orElse(Aggregation(query).headOption)
val planned =
plannedOption.getOrElse(
Expand All @@ -51,6 +53,8 @@ class PlannerSuite extends SparkFunSuite with SQLTestUtils {
}

test("unions are collapsed") {
val _ctx = ctx
import _ctx.planner._
val query = testData.unionAll(testData).unionAll(testData).logicalPlan
val planned = BasicOperators(query).head
val logicalUnions = query collect { case u: logical.Union => u }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ class RowFormatConvertersSuite extends SparkPlanTest {
}

test("SPARK-9683: copy UTF8String when convert unsafe array/map to safe") {
SparkPlan.currentContext.set(TestSQLContext)
SparkPlan.currentContext.set(ctx)
val schema = ArrayType(StringType)
val rows = (1 to 100).map { i =>
InternalRow(new GenericArrayData(Array[Any](UTF8String.fromString(i.toString))))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import org.scalatest.Matchers
import org.apache.spark.sql.catalyst.expressions.{UnsafeRow, UnsafeProjection}
import org.apache.spark.{TaskContextImpl, TaskContext, SparkFunSuite}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.memory.{ExecutorMemoryManager, MemoryAllocator, TaskMemoryManager}
import org.apache.spark.unsafe.types.UTF8String
Expand All @@ -39,7 +39,7 @@ import org.apache.spark.unsafe.types.UTF8String
class UnsafeFixedWidthAggregationMapSuite
extends SparkFunSuite
with Matchers
with SharedSQLContext {
with SQLTestUtils {

import UnsafeFixedWidthAggregationMap._

Expand Down Expand Up @@ -233,8 +233,6 @@ class UnsafeFixedWidthAggregationMapSuite
}

testWithMemoryLeakDetection("test external sorting with an empty map") {
// Calling this make sure we have block manager and everything else setup.
TestSQLContext

val map = new UnsafeFixedWidthAggregationMap(
emptyAggregationBuffer,
Expand Down Expand Up @@ -282,8 +280,6 @@ class UnsafeFixedWidthAggregationMapSuite
}

testWithMemoryLeakDetection("test external sorting with empty records") {
// Calling this make sure we have block manager and everything else setup.
TestSQLContext

// Memory consumption in the beginning of the task.
val initialMemoryConsumption = shuffleMemoryManager.getMemoryConsumptionForThisTask()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1056,7 +1056,7 @@ class JsonSuite extends QueryTest with SQLTestUtils with TestJsonData {
Some(singleRow),
1.0,
Some(StructType(StructField("b", IntegerType, true) :: Nil)),
None, None)(çtx)
None, None)(ctx)
val logicalRelation3 = LogicalRelation(relation3)

assert(relation0 !== relation1)
Expand All @@ -1081,14 +1081,14 @@ class JsonSuite extends QueryTest with SQLTestUtils with TestJsonData {
.map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path)

val d1 = ResolvedDataSource(
context,
ctx,
userSpecifiedSchema = None,
partitionColumns = Array.empty[String],
provider = classOf[DefaultSource].getCanonicalName,
options = Map("path" -> path))

val d2 = ResolvedDataSource(
context,
ctx,
userSpecifiedSchema = None,
partitionColumns = Array.empty[String],
provider = classOf[DefaultSource].getCanonicalName,
Expand Down Expand Up @@ -1154,11 +1154,12 @@ class JsonSuite extends QueryTest with SQLTestUtils with TestJsonData {
"abd")

ctx.read.json(root.getAbsolutePath).registerTempTable("test_myjson_with_part")
checkAnswer(
sql("SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abc'"), Row(4))
checkAnswer(
sql("SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abd'"), Row(5))
checkAnswer(sql("SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
checkAnswer(ctx.sql(
"SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abc'"), Row(4))
checkAnswer(ctx.sql(
"SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abd'"), Row(5))
checkAnswer(ctx.sql(
"SELECT count(a) FROM test_myjson_with_part where d1 = 1"), Row(9))
})
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,14 @@ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._
import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.test.TestSQLContext
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.util.Utils


class SQLMetricsSuite extends SparkFunSuite {
class SQLMetricsSuite extends SparkFunSuite with SQLTestUtils {

test("LongSQLMetric should not box Long") {
val l = SQLMetrics.createLongMetric(TestSQLContext.sparkContext, "long")
val l = SQLMetrics.createLongMetric(ctx.sparkContext, "long")
val f = () => { l += 1L }
BoxingFinder.getClassReader(f.getClass).foreach { cl =>
val boxingFinder = new BoxingFinder()
Expand All @@ -43,7 +43,7 @@ class SQLMetricsSuite extends SparkFunSuite {

test("Normal accumulator should do boxing") {
// We need this test to make sure BoxingFinder works.
val l = TestSQLContext.sparkContext.accumulator(0L)
val l = ctx.sparkContext.accumulator(0L)
val f = () => { l += 1L }
BoxingFinder.getClassReader(f.getClass).foreach { cl =>
val boxingFinder = new BoxingFinder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,12 @@ import org.apache.spark.sql.execution.metric.LongSQLMetricValue
import org.apache.spark.scheduler._
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.sql.execution.SQLExecution
import org.apache.spark.sql.test.TestSQLContext
import org.apache.spark.sql.test.SQLTestUtils

class SQLListenerSuite extends SparkFunSuite {
class SQLListenerSuite extends SparkFunSuite with SQLTestUtils {
import testImplicits._

private def createTestDataFrame: DataFrame = {
import TestSQLContext.implicits._
Seq(
(1, 1),
(2, 2)
Expand Down Expand Up @@ -74,7 +74,7 @@ class SQLListenerSuite extends SparkFunSuite {
}

test("basic") {
val listener = new SQLListener(TestSQLContext)
val listener = new SQLListener(ctx)
val executionId = 0
val df = createTestDataFrame
val accumulatorIds =
Expand Down Expand Up @@ -212,7 +212,7 @@ class SQLListenerSuite extends SparkFunSuite {
}

test("onExecutionEnd happens before onJobEnd(JobSucceeded)") {
val listener = new SQLListener(TestSQLContext)
val listener = new SQLListener(ctx)
val executionId = 0
val df = createTestDataFrame
listener.onExecutionStart(
Expand Down Expand Up @@ -241,7 +241,7 @@ class SQLListenerSuite extends SparkFunSuite {
}

test("onExecutionEnd happens before multiple onJobEnd(JobSucceeded)s") {
val listener = new SQLListener(TestSQLContext)
val listener = new SQLListener(ctx)
val executionId = 0
val df = createTestDataFrame
listener.onExecutionStart(
Expand Down Expand Up @@ -281,7 +281,7 @@ class SQLListenerSuite extends SparkFunSuite {
}

test("onExecutionEnd happens before onJobEnd(JobFailed)") {
val listener = new SQLListener(TestSQLContext)
val listener = new SQLListener(ctx)
val executionId = 0
val df = createTestDataFrame
listener.onExecutionStart(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll {

test("Caching") {
// write something to the jsonTable
sql(
caseInsensitiveContext.sql(
s"""
|INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt
""".stripMargin)
Expand Down Expand Up @@ -230,7 +230,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll {
|INSERT OVERWRITE TABLE jsonTable SELECT a * 2, b FROM jt
""".stripMargin)
// jsonTable should be recached.
assertCached(sql("SELECT * FROM jsonTable"))
assertCached(caseInsensitiveContext.sql("SELECT * FROM jsonTable"))
// TODO we need to invalidate the cached data in InsertIntoHadoopFsRelation
// // The cached data is the new data.
// checkAnswer(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,21 +19,21 @@ package org.apache.spark.sql.sources

import org.apache.spark.sql.{Row, QueryTest}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.TestSQLContext
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.util.Utils

class PartitionedWriteSuite extends QueryTest {
import TestSQLContext.implicits._
class PartitionedWriteSuite extends QueryTest with SQLTestUtils {
import testImplicits._

test("write many partitions") {
val path = Utils.createTempDir()
path.delete()

val df = TestSQLContext.range(100).select($"id", lit(1).as("data"))
val df = ctx.range(100).select($"id", lit(1).as("data"))
df.write.partitionBy("id").save(path.getCanonicalPath)

checkAnswer(
TestSQLContext.read.load(path.getCanonicalPath),
ctx.read.load(path.getCanonicalPath),
(0 to 99).map(Row(1, _)).toSeq)

Utils.deleteRecursively(path)
Expand All @@ -43,12 +43,12 @@ class PartitionedWriteSuite extends QueryTest {
val path = Utils.createTempDir()
path.delete()

val base = TestSQLContext.range(100)
val base = ctx.range(100)
val df = base.unionAll(base).select($"id", lit(1).as("data"))
df.write.partitionBy("id").save(path.getCanonicalPath)

checkAnswer(
TestSQLContext.read.load(path.getCanonicalPath),
ctx.read.load(path.getCanonicalPath),
(0 to 99).map(Row(1, _)).toSeq ++ (0 to 99).map(Row(1, _)).toSeq)

Utils.deleteRecursively(path)
Expand Down