Skip to content

Commit

Permalink
Limit kinesis test dependencies
Browse files Browse the repository at this point in the history
KinesisReceiverSuite really doesn't need to depend on
TestSuiteBase. The issue is that TestSuiteBase transitively
depends on Spark core tests (SparkFunSuite), but there is no simple
way in maven to express transitive test dependencies. To simplify
the build structure of this module, we should just have
KinesisReceiverSuite extend directly from FunSuite.
  • Loading branch information
Andrew Or committed Jul 2, 2015
1 parent f24e09c commit ca3d5d4
Showing 1 changed file with 12 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -57,24 +62,23 @@ 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]
checkpointStateMock = mock[KinesisCheckpointState]
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,
checkpointStateMock, currentClockMock)
}

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),
Expand Down

0 comments on commit ca3d5d4

Please sign in to comment.