aboutsummaryrefslogtreecommitdiff
path: root/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala
blob: afb55c84f81fe8a1568f7e66bd182842edac5fa5 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.streaming.kinesis

import scala.collection.mutable
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.Random

import com.amazonaws.regions.RegionUtils
import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
import com.amazonaws.services.kinesis.model.Record
import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
import org.scalatest.Matchers._
import org.scalatest.concurrent.Eventually

import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.{StorageLevel, StreamBlockId}
import org.apache.spark.streaming._
import org.apache.spark.streaming.dstream.ReceiverInputDStream
import org.apache.spark.streaming.kinesis.KinesisTestUtils._
import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult
import org.apache.spark.streaming.scheduler.ReceivedBlockInfo
import org.apache.spark.util.Utils

abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFunSuite
  with Eventually with BeforeAndAfter with BeforeAndAfterAll {

  // This is the name that KCL will use to save metadata to DynamoDB
  private val appName = s"KinesisStreamSuite-${math.abs(Random.nextLong())}"
  private val batchDuration = Seconds(1)

  // Dummy parameters for API testing
  private val dummyEndpointUrl = defaultEndpointUrl
  private val dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl)
  private val dummyAWSAccessKey = "dummyAccessKey"
  private val dummyAWSSecretKey = "dummySecretKey"

  private var testUtils: KinesisTestUtils = null
  private var ssc: StreamingContext = null
  private var sc: SparkContext = null

  override def beforeAll(): Unit = {
    val conf = new SparkConf()
      .setMaster("local[4]")
      .setAppName("KinesisStreamSuite") // Setting Spark app name to Kinesis app name
    sc = new SparkContext(conf)

    runIfTestsEnabled("Prepare KinesisTestUtils") {
      testUtils = new KPLBasedKinesisTestUtils()
      testUtils.createStream()
    }
  }

  override def afterAll(): Unit = {
    if (ssc != null) {
      ssc.stop()
    }
    if (sc != null) {
      sc.stop()
    }
    if (testUtils != null) {
      // Delete the Kinesis stream as well as the DynamoDB table generated by
      // Kinesis Client Library when consuming the stream
      testUtils.deleteStream()
      testUtils.deleteDynamoDBTable(appName)
    }
  }

  before {
    ssc = new StreamingContext(sc, batchDuration)
  }

  after {
    if (ssc != null) {
      ssc.stop(stopSparkContext = false)
      ssc = null
    }
    if (testUtils != null) {
      testUtils.deleteDynamoDBTable(appName)
    }
  }

  test("KinesisUtils API") {
    val kinesisStream1 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
      dummyEndpointUrl, dummyRegionName,
      InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2)
    val kinesisStream2 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
      dummyEndpointUrl, dummyRegionName,
      InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2,
      dummyAWSAccessKey, dummyAWSSecretKey)
  }

  test("RDD generation") {
    val inputStream = KinesisUtils.createStream(ssc, appName, "dummyStream",
      dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, Seconds(2),
      StorageLevel.MEMORY_AND_DISK_2, dummyAWSAccessKey, dummyAWSSecretKey)
    assert(inputStream.isInstanceOf[KinesisInputDStream[Array[Byte]]])

    val kinesisStream = inputStream.asInstanceOf[KinesisInputDStream[Array[Byte]]]
    val time = Time(1000)

    // Generate block info data for testing
    val seqNumRanges1 = SequenceNumberRanges(
      SequenceNumberRange("fakeStream", "fakeShardId", "xxx", "yyy", 67))
    val blockId1 = StreamBlockId(kinesisStream.id, 123)
    val blockInfo1 = ReceivedBlockInfo(
      0, None, Some(seqNumRanges1), new BlockManagerBasedStoreResult(blockId1, None))

    val seqNumRanges2 = SequenceNumberRanges(
      SequenceNumberRange("fakeStream", "fakeShardId", "aaa", "bbb", 89))
    val blockId2 = StreamBlockId(kinesisStream.id, 345)
    val blockInfo2 = ReceivedBlockInfo(
      0, None, Some(seqNumRanges2), new BlockManagerBasedStoreResult(blockId2, None))

    // Verify that the generated KinesisBackedBlockRDD has the all the right information
    val blockInfos = Seq(blockInfo1, blockInfo2)
    val nonEmptyRDD = kinesisStream.createBlockRDD(time, blockInfos)
    nonEmptyRDD shouldBe a [KinesisBackedBlockRDD[_]]
    val kinesisRDD = nonEmptyRDD.asInstanceOf[KinesisBackedBlockRDD[_]]
    assert(kinesisRDD.regionName === dummyRegionName)
    assert(kinesisRDD.endpointUrl === dummyEndpointUrl)
    assert(kinesisRDD.retryTimeoutMs === batchDuration.milliseconds)
    assert(kinesisRDD.kinesisCredsProvider === BasicCredentialsProvider(
      awsAccessKeyId = dummyAWSAccessKey,
      awsSecretKey = dummyAWSSecretKey))
    assert(nonEmptyRDD.partitions.size === blockInfos.size)
    nonEmptyRDD.partitions.foreach { _ shouldBe a [KinesisBackedBlockRDDPartition] }
    val partitions = nonEmptyRDD.partitions.map {
      _.asInstanceOf[KinesisBackedBlockRDDPartition] }.toSeq
    assert(partitions.map { _.seqNumberRanges } === Seq(seqNumRanges1, seqNumRanges2))
    assert(partitions.map { _.blockId } === Seq(blockId1, blockId2))
    assert(partitions.forall { _.isBlockIdValid === true })

    // Verify that KinesisBackedBlockRDD is generated even when there are no blocks
    val emptyRDD = kinesisStream.createBlockRDD(time, Seq.empty)
    // Verify it's KinesisBackedBlockRDD[_] rather than KinesisBackedBlockRDD[Array[Byte]], because
    // the type parameter will be erased at runtime
    emptyRDD shouldBe a [KinesisBackedBlockRDD[_]]
    emptyRDD.partitions shouldBe empty

    // Verify that the KinesisBackedBlockRDD has isBlockValid = false when blocks are invalid
    blockInfos.foreach { _.setBlockIdInvalid() }
    kinesisStream.createBlockRDD(time, blockInfos).partitions.foreach { partition =>
      assert(partition.asInstanceOf[KinesisBackedBlockRDDPartition].isBlockIdValid === false)
    }
  }


  /**
   * Test the stream by sending data to a Kinesis stream and receiving from it.
   * This test is not run by default as it requires AWS credentials that the test
   * environment may not have. Even if there is AWS credentials available, the user
   * may not want to run these tests to avoid the Kinesis costs. To enable this test,
   * you must have AWS credentials available through the default AWS provider chain,
   * and you have to set the system environment variable RUN_KINESIS_TESTS=1 .
   */
  testIfEnabled("basic operation") {
    val awsCredentials = KinesisTestUtils.getAWSCredentials()
    val stream = KinesisUtils.createStream(ssc, appName, testUtils.streamName,
      testUtils.endpointUrl, testUtils.regionName, InitialPositionInStream.LATEST,
      Seconds(10), StorageLevel.MEMORY_ONLY,
      awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey)

    val collected = new mutable.HashSet[Int]
    stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd =>
      collected.synchronized {
        collected ++= rdd.collect()
        logInfo("Collected = " + collected.mkString(", "))
      }
    }
    ssc.start()

    val testData = 1 to 10
    eventually(timeout(120 seconds), interval(10 second)) {
      testUtils.pushData(testData, aggregateTestData)
      assert(collected.synchronized { collected === testData.toSet },
        "\nData received does not match data sent")
    }
    ssc.stop(stopSparkContext = false)
  }

  testIfEnabled("custom message handling") {
    val awsCredentials = KinesisTestUtils.getAWSCredentials()
    def addFive(r: Record): Int = JavaUtils.bytesToString(r.getData).toInt + 5
    val stream = KinesisUtils.createStream(ssc, appName, testUtils.streamName,
      testUtils.endpointUrl, testUtils.regionName, InitialPositionInStream.LATEST,
      Seconds(10), StorageLevel.MEMORY_ONLY, addFive(_),
      awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey)

    stream shouldBe a [ReceiverInputDStream[_]]

    val collected = new mutable.HashSet[Int]
    stream.foreachRDD { rdd =>
      collected.synchronized {
        collected ++= rdd.collect()
        logInfo("Collected = " + collected.mkString(", "))
      }
    }
    ssc.start()

    val testData = 1 to 10
    eventually(timeout(120 seconds), interval(10 second)) {
      testUtils.pushData(testData, aggregateTestData)
      val modData = testData.map(_ + 5)
      assert(collected.synchronized { collected === modData.toSet },
        "\nData received does not match data sent")
    }
    ssc.stop(stopSparkContext = false)
  }

  testIfEnabled("split and merge shards in a stream") {
    // Since this test tries to split and merge shards in a stream, we create another
    // temporary stream and then remove it when finished.
    val localAppName = s"KinesisStreamSuite-${math.abs(Random.nextLong())}"
    val localTestUtils = new KPLBasedKinesisTestUtils(1)
    localTestUtils.createStream()
    try {
      val awsCredentials = KinesisTestUtils.getAWSCredentials()
      val stream = KinesisUtils.createStream(ssc, localAppName, localTestUtils.streamName,
        localTestUtils.endpointUrl, localTestUtils.regionName, InitialPositionInStream.LATEST,
        Seconds(10), StorageLevel.MEMORY_ONLY,
        awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey)

      val collected = new mutable.HashSet[Int]
      stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd =>
        collected.synchronized {
          collected ++= rdd.collect()
          logInfo("Collected = " + collected.mkString(", "))
        }
      }
      ssc.start()

      val testData1 = 1 to 10
      val testData2 = 11 to 20
      val testData3 = 21 to 30

      eventually(timeout(60 seconds), interval(10 second)) {
        localTestUtils.pushData(testData1, aggregateTestData)
        assert(collected.synchronized { collected === testData1.toSet },
          "\nData received does not match data sent")
      }

      val shardToSplit = localTestUtils.getShards().head
      localTestUtils.splitShard(shardToSplit.getShardId)
      val (splitOpenShards, splitCloseShards) = localTestUtils.getShards().partition { shard =>
        shard.getSequenceNumberRange.getEndingSequenceNumber == null
      }

      // We should have one closed shard and two open shards
      assert(splitCloseShards.size == 1)
      assert(splitOpenShards.size == 2)

      eventually(timeout(60 seconds), interval(10 second)) {
        localTestUtils.pushData(testData2, aggregateTestData)
        assert(collected.synchronized { collected === (testData1 ++ testData2).toSet },
          "\nData received does not match data sent after splitting a shard")
      }

      val Seq(shardToMerge, adjShard) = splitOpenShards
      localTestUtils.mergeShard(shardToMerge.getShardId, adjShard.getShardId)
      val (mergedOpenShards, mergedCloseShards) = localTestUtils.getShards().partition { shard =>
        shard.getSequenceNumberRange.getEndingSequenceNumber == null
      }

      // We should have three closed shards and one open shard
      assert(mergedCloseShards.size == 3)
      assert(mergedOpenShards.size == 1)

      eventually(timeout(60 seconds), interval(10 second)) {
        localTestUtils.pushData(testData3, aggregateTestData)
        assert(collected.synchronized { collected === (testData1 ++ testData2 ++ testData3).toSet },
          "\nData received does not match data sent after merging shards")
      }
    } finally {
      ssc.stop(stopSparkContext = false)
      localTestUtils.deleteStream()
      localTestUtils.deleteDynamoDBTable(localAppName)
    }
  }

  testIfEnabled("failure recovery") {
    val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName)
    val checkpointDir = Utils.createTempDir().getAbsolutePath

    ssc = new StreamingContext(sc, Milliseconds(1000))
    ssc.checkpoint(checkpointDir)

    val awsCredentials = KinesisTestUtils.getAWSCredentials()
    val collectedData = new mutable.HashMap[Time, (Array[SequenceNumberRanges], Seq[Int])]

    val kinesisStream = KinesisUtils.createStream(ssc, appName, testUtils.streamName,
      testUtils.endpointUrl, testUtils.regionName, InitialPositionInStream.LATEST,
      Seconds(10), StorageLevel.MEMORY_ONLY,
      awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey)

    // Verify that the generated RDDs are KinesisBackedBlockRDDs, and collect the data in each batch
    kinesisStream.foreachRDD((rdd: RDD[Array[Byte]], time: Time) => {
      val kRdd = rdd.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]]
      val data = rdd.map { bytes => new String(bytes).toInt }.collect().toSeq
      collectedData.synchronized {
        collectedData(time) = (kRdd.arrayOfseqNumberRanges, data)
      }
    })

    ssc.remember(Minutes(60)) // remember all the batches so that they are all saved in checkpoint
    ssc.start()

    def numBatchesWithData: Int =
      collectedData.synchronized { collectedData.count(_._2._2.nonEmpty) }

    def isCheckpointPresent: Boolean = Checkpoint.getCheckpointFiles(checkpointDir).nonEmpty

    // Run until there are at least 10 batches with some data in them
    // If this times out because numBatchesWithData is empty, then its likely that foreachRDD
    // function failed with exceptions, and nothing got added to `collectedData`
    eventually(timeout(2 minutes), interval(1 seconds)) {
      testUtils.pushData(1 to 5, aggregateTestData)
      assert(isCheckpointPresent && numBatchesWithData > 10)
    }
    ssc.stop(stopSparkContext = true)  // stop the SparkContext so that the blocks are not reused

    // Restart the context from checkpoint and verify whether the
    logInfo("Restarting from checkpoint")
    ssc = new StreamingContext(checkpointDir)
    ssc.start()
    val recoveredKinesisStream = ssc.graph.getInputStreams().head

    // Verify that the recomputed RDDs are KinesisBackedBlockRDDs with the same sequence ranges
    // and return the same data
    collectedData.synchronized {
      val times = collectedData.keySet
      times.foreach { time =>
        val (arrayOfSeqNumRanges, data) = collectedData(time)
        val rdd = recoveredKinesisStream.getOrCompute(time).get.asInstanceOf[RDD[Array[Byte]]]
        rdd shouldBe a[KinesisBackedBlockRDD[_]]

        // Verify the recovered sequence ranges
        val kRdd = rdd.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]]
        assert(kRdd.arrayOfseqNumberRanges.size === arrayOfSeqNumRanges.size)
        arrayOfSeqNumRanges.zip(kRdd.arrayOfseqNumberRanges).foreach { case (expected, found) =>
          assert(expected.ranges.toSeq === found.ranges.toSeq)
        }

        // Verify the recovered data
        assert(rdd.map { bytes => new String(bytes).toInt }.collect().toSeq === data)
      }
    }
    ssc.stop()
  }
}

class WithAggregationKinesisStreamSuite extends KinesisStreamTests(aggregateTestData = true)

class WithoutAggregationKinesisStreamSuite extends KinesisStreamTests(aggregateTestData = false)