Skip to content

Commit 7e3e9af

Browse files
marmbrusrxin
authored andcommitted
[SQL] Update SparkSQL and ScalaTest in branch-1.0 to match master.
#511 and #863 got left out of branch-1.0 since we were really close to the release. Now that they have been tested a little I see no reason to leave them out. Author: Michael Armbrust <[email protected]> Author: witgo <[email protected]> Closes #1078 from marmbrus/branch-1.0 and squashes the following commits: 22be674 [witgo] [SPARK-1841]: update scalatest to version 2.1.5 fc8fc79 [Michael Armbrust] Include #1071 as well. c5d0adf [Michael Armbrust] Update SparkSQL in branch-1.0 to match master.
1 parent 00b4317 commit 7e3e9af

File tree

30 files changed

+1039
-254
lines changed

30 files changed

+1039
-254
lines changed

core/pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -235,7 +235,7 @@
235235
</dependency>
236236
<dependency>
237237
<groupId>org.easymock</groupId>
238-
<artifactId>easymock</artifactId>
238+
<artifactId>easymockclassextension</artifactId>
239239
<scope>test</scope>
240240
</dependency>
241241
<dependency>

core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import scala.language.postfixOps
2525
import scala.util.Random
2626

2727
import org.scalatest.{BeforeAndAfter, FunSuite}
28-
import org.scalatest.concurrent.Eventually
28+
import org.scalatest.concurrent.{PatienceConfiguration, Eventually}
2929
import org.scalatest.concurrent.Eventually._
3030
import org.scalatest.time.SpanSugar._
3131

@@ -76,7 +76,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo
7676
tester.assertCleanup()
7777

7878
// Verify that shuffles can be re-executed after cleaning up
79-
assert(rdd.collect().toList === collected)
79+
assert(rdd.collect().toList.equals(collected))
8080
}
8181

8282
test("cleanup broadcast") {
@@ -285,7 +285,7 @@ class CleanerTester(
285285
sc.cleaner.get.attachListener(cleanerListener)
286286

287287
/** Assert that all the stuff has been cleaned up */
288-
def assertCleanup()(implicit waitTimeout: Eventually.Timeout) {
288+
def assertCleanup()(implicit waitTimeout: PatienceConfiguration.Timeout) {
289289
try {
290290
eventually(waitTimeout, interval(100 millis)) {
291291
assert(isAllCleanedUp)

core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,11 +23,11 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll {
2323

2424
// This test suite should run all tests in ShuffleSuite with Netty shuffle mode.
2525

26-
override def beforeAll(configMap: Map[String, Any]) {
26+
override def beforeAll() {
2727
System.setProperty("spark.shuffle.use.netty", "true")
2828
}
2929

30-
override def afterAll(configMap: Map[String, Any]) {
30+
override def afterAll() {
3131
System.setProperty("spark.shuffle.use.netty", "false")
3232
}
3333
}

core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -275,8 +275,9 @@ class RDDSuite extends FunSuite with SharedSparkContext {
275275

276276
// we can optionally shuffle to keep the upstream parallel
277277
val coalesced5 = data.coalesce(1, shuffle = true)
278-
assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] !=
279-
null)
278+
val isEquals = coalesced5.dependencies.head.rdd.dependencies.head.rdd.
279+
asInstanceOf[ShuffledRDD[_, _, _]] != null
280+
assert(isEquals)
280281

281282
// when shuffling, we can increase the number of partitions
282283
val coalesced6 = data.coalesce(20, shuffle = true)

core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import scala.language.reflectiveCalls
2323

2424
import akka.actor._
2525
import akka.testkit.{ImplicitSender, TestKit, TestActorRef}
26-
import org.scalatest.{BeforeAndAfter, FunSuite}
26+
import org.scalatest.{BeforeAndAfter, FunSuiteLike}
2727

2828
import org.apache.spark._
2929
import org.apache.spark.rdd.RDD
@@ -37,7 +37,7 @@ class BuggyDAGEventProcessActor extends Actor {
3737
}
3838
}
3939

40-
class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuite
40+
class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike
4141
with ImplicitSender with BeforeAndAfter with LocalSparkContext {
4242

4343
val conf = new SparkConf

core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,8 @@ class TimeStampedHashMapSuite extends FunSuite {
105105
map("k1") = strongRef
106106
map("k2") = "v2"
107107
map("k3") = "v3"
108-
assert(map("k1") === strongRef)
108+
val isEquals = map("k1") == strongRef
109+
assert(isEquals)
109110

110111
// clear strong reference to "k1"
111112
strongRef = null

pom.xml

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -458,25 +458,31 @@
458458
<dependency>
459459
<groupId>org.scalatest</groupId>
460460
<artifactId>scalatest_${scala.binary.version}</artifactId>
461-
<version>1.9.1</version>
461+
<version>2.1.5</version>
462462
<scope>test</scope>
463463
</dependency>
464464
<dependency>
465465
<groupId>org.easymock</groupId>
466-
<artifactId>easymock</artifactId>
466+
<artifactId>easymockclassextension</artifactId>
467467
<version>3.1</version>
468468
<scope>test</scope>
469469
</dependency>
470470
<dependency>
471471
<groupId>org.mockito</groupId>
472472
<artifactId>mockito-all</artifactId>
473-
<version>1.8.5</version>
473+
<version>1.9.0</version>
474474
<scope>test</scope>
475475
</dependency>
476476
<dependency>
477477
<groupId>org.scalacheck</groupId>
478478
<artifactId>scalacheck_${scala.binary.version}</artifactId>
479-
<version>1.10.0</version>
479+
<version>1.11.3</version>
480+
<scope>test</scope>
481+
</dependency>
482+
<dependency>
483+
<groupId>junit</groupId>
484+
<artifactId>junit</artifactId>
485+
<version>4.10</version>
480486
<scope>test</scope>
481487
</dependency>
482488
<dependency>
@@ -778,6 +784,7 @@
778784
<arg>-unchecked</arg>
779785
<arg>-deprecation</arg>
780786
<arg>-feature</arg>
787+
<arg>-language:postfixOps</arg>
781788
</args>
782789
<jvmArgs>
783790
<jvmArg>-Xms1024m</jvmArg>

project/SparkBuild.scala

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -270,16 +270,17 @@ object SparkBuild extends Build {
270270
*/
271271

272272
libraryDependencies ++= Seq(
273-
"io.netty" % "netty-all" % "4.0.17.Final",
274-
"org.eclipse.jetty" % "jetty-server" % jettyVersion,
275-
"org.eclipse.jetty" % "jetty-util" % jettyVersion,
276-
"org.eclipse.jetty" % "jetty-plus" % jettyVersion,
277-
"org.eclipse.jetty" % "jetty-security" % jettyVersion,
278-
"org.scalatest" %% "scalatest" % "1.9.1" % "test",
279-
"org.scalacheck" %% "scalacheck" % "1.10.0" % "test",
280-
"com.novocode" % "junit-interface" % "0.10" % "test",
281-
"org.easymock" % "easymock" % "3.1" % "test",
282-
"org.mockito" % "mockito-all" % "1.8.5" % "test"
273+
"io.netty" % "netty-all" % "4.0.17.Final",
274+
"org.eclipse.jetty" % "jetty-server" % jettyVersion,
275+
"org.eclipse.jetty" % "jetty-util" % jettyVersion,
276+
"org.eclipse.jetty" % "jetty-plus" % jettyVersion,
277+
"org.eclipse.jetty" % "jetty-security" % jettyVersion,
278+
"org.scalatest" %% "scalatest" % "2.1.5" % "test",
279+
"org.scalacheck" %% "scalacheck" % "1.11.3" % "test",
280+
"com.novocode" % "junit-interface" % "0.10" % "test",
281+
"org.easymock" % "easymockclassextension" % "3.1" % "test",
282+
"org.mockito" % "mockito-all" % "1.9.0" % "test",
283+
"junit" % "junit" % "4.10" % "test"
283284
),
284285

285286
testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"),
@@ -476,7 +477,6 @@ object SparkBuild extends Build {
476477
// this non-deterministically. TODO: FIX THIS.
477478
parallelExecution in Test := false,
478479
libraryDependencies ++= Seq(
479-
"org.scalatest" %% "scalatest" % "1.9.1" % "test",
480480
"com.typesafe" %% "scalalogging-slf4j" % "1.0.1"
481481
)
482482
)

repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -57,12 +57,14 @@ class ReplSuite extends FunSuite {
5757
}
5858

5959
def assertContains(message: String, output: String) {
60-
assert(output.contains(message),
60+
val isContain = output.contains(message)
61+
assert(isContain,
6162
"Interpreter output did not contain '" + message + "':\n" + output)
6263
}
6364

6465
def assertDoesNotContain(message: String, output: String) {
65-
assert(!output.contains(message),
66+
val isContain = output.contains(message)
67+
assert(!isContain,
6668
"Interpreter output contained '" + message + "':\n" + output)
6769
}
6870

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.catalyst.plans.logical
1919

20-
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute}
20+
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference}
2121
import org.apache.spark.sql.catalyst.types.StringType
2222

2323
/**
@@ -26,35 +26,37 @@ import org.apache.spark.sql.catalyst.types.StringType
2626
*/
2727
abstract class Command extends LeafNode {
2828
self: Product =>
29-
def output: Seq[Attribute] = Seq.empty // TODO: SPARK-2081 should fix this
29+
def output: Seq[Attribute] = Seq.empty
3030
}
3131

3232
/**
3333
* Returned for commands supported by a given parser, but not catalyst. In general these are DDL
3434
* commands that are passed directly to another system.
3535
*/
36-
case class NativeCommand(cmd: String) extends Command
36+
case class NativeCommand(cmd: String) extends Command {
37+
override def output =
38+
Seq(BoundReference(0, AttributeReference("result", StringType, nullable = false)()))
39+
}
3740

3841
/**
3942
* Commands of the form "SET (key) (= value)".
4043
*/
4144
case class SetCommand(key: Option[String], value: Option[String]) extends Command {
4245
override def output = Seq(
43-
AttributeReference("key", StringType, nullable = false)(),
44-
AttributeReference("value", StringType, nullable = false)()
45-
)
46+
BoundReference(0, AttributeReference("key", StringType, nullable = false)()),
47+
BoundReference(1, AttributeReference("value", StringType, nullable = false)()))
4648
}
4749

4850
/**
4951
* Returned by a parser when the users only wants to see what query plan would be executed, without
5052
* actually performing the execution.
5153
*/
5254
case class ExplainCommand(plan: LogicalPlan) extends Command {
53-
override def output = Seq(AttributeReference("plan", StringType, nullable = false)())
55+
override def output =
56+
Seq(BoundReference(0, AttributeReference("plan", StringType, nullable = false)()))
5457
}
5558

5659
/**
5760
* Returned for the "CACHE TABLE tableName" and "UNCACHE TABLE tableName" command.
5861
*/
5962
case class CacheCommand(tableName: String, doCache: Boolean) extends Command
60-

0 commit comments

Comments
 (0)