aboutsummaryrefslogtreecommitdiff
path: root/extras
diff options
context:
space:
mode:
Diffstat (limited to 'extras')
-rw-r--r--extras/kinesis-asl/pom.xml7
-rw-r--r--extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala20
2 files changed, 12 insertions, 15 deletions
diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml
index c242e7a57b..5289073eb4 100644
--- a/extras/kinesis-asl/pom.xml
+++ b/extras/kinesis-asl/pom.xml
@@ -42,13 +42,6 @@
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_${scala.binary.version}</artifactId>
- <version>${project.version}</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala
index 2103dca6b7..6c26262483 100644
--- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala
+++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala
@@ -26,18 +26,23 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionIn
import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason
import com.amazonaws.services.kinesis.model.Record
import org.mockito.Mockito._
-import org.scalatest.{BeforeAndAfter, Matchers}
+// scalastyle:off
+// To avoid introducing a dependency on Spark core tests, simply use scalatest's FunSuite
+// here instead of our own SparkFunSuite. Introducing the dependency has caused problems
+// in the past (SPARK-8781) that are complicated by bugs in the maven shade plugin (MSHADE-148).
+import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
import org.scalatest.mock.MockitoSugar
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext, TestSuiteBase}
+import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext}
import org.apache.spark.util.{Clock, ManualClock, Utils}
/**
* Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor
*/
-class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter
- with MockitoSugar {
+class KinesisReceiverSuite extends FunSuite with Matchers with BeforeAndAfter
+ with MockitoSugar {
+// scalastyle:on
val app = "TestKinesisReceiver"
val stream = "mySparkStream"
@@ -57,7 +62,7 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft
var checkpointStateMock: KinesisCheckpointState = _
var currentClockMock: Clock = _
- override def beforeFunction(): Unit = {
+ before {
receiverMock = mock[KinesisReceiver]
checkpointerMock = mock[IRecordProcessorCheckpointer]
checkpointClockMock = mock[ManualClock]
@@ -65,8 +70,7 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft
currentClockMock = mock[Clock]
}
- override def afterFunction(): Unit = {
- super.afterFunction()
+ after {
// Since this suite was originally written using EasyMock, add this to preserve the old
// mocking semantics (see SPARK-5735 for more details)
verifyNoMoreInteractions(receiverMock, checkpointerMock, checkpointClockMock,
@@ -74,7 +78,7 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft
}
test("KinesisUtils API") {
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext("local[2]", getClass.getSimpleName, Seconds(1))
// Tests the API, does not actually test data receiving
val kinesisStream1 = KinesisUtils.createStream(ssc, "mySparkStream",
"https://kinesis.us-west-2.amazonaws.com", Seconds(2),