Skip to content

Commit

Permalink
[SPARK-9217] [STREAMING] Make the kinesis receiver reliable by record…
Browse files Browse the repository at this point in the history
…ing sequence numbers

This PR is the second one in the larger issue of making the Kinesis integration reliable and provide WAL-free at-least once guarantee. It is based on the design doc - https://docs.google.com/document/d/1k0dl270EnK7uExrsCE7jYw7PYx0YC935uBcxn3p0f58/edit

In this PR, I have updated the Kinesis Receiver to do the following.
- Control the block generation, by creating its own BlockGenerator with own callback methods and using it to keep track of the ranges of sequence numbers that go into each block.
- More specifically, as the KinesisRecordProcessor provides small batches of records, the records are atomically inserted into the block (that is, either the whole batch is in the block, or not). Accordingly the sequence number range of the batch is recorded. Since there may be many batches added to a block, the receiver tracks all the range of sequence numbers that is added to a block.
- When the block is ready to be pushed, the block is pushed and the ranges are reported as metadata of the block. In addition, the ranges are used to find out the latest sequence number for each shard that can be checkpointed through the DynamoDB.
- Periodically, each KinesisRecordProcessor checkpoints the latest successfully stored sequence number for it own shard.
- The array of ranges in the block metadata is used to create KinesisBackedBlockRDDs. The ReceiverInputDStream has been slightly refactored to allow the creation of KinesisBackedBlockRDDs instead of the WALBackedBlockRDDs.

Things to be done
- [x] Add new test to verify that the sequence numbers are recovered.

Author: Tathagata Das <[email protected]>

Closes #7825 from tdas/kinesis-receiver and squashes the following commits:

2159be9 [Tathagata Das] Fixed bug
569be83 [Tathagata Das] Fix scala style issue
bf31e22 [Tathagata Das] Added more documentation to make the kinesis test endpoint more configurable
3ad8361 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into kinesis-receiver
c693a63 [Tathagata Das] Removed unnecessary constructor params from KinesisTestUtils
e1f1d0a [Tathagata Das] Addressed PR comments
b9fa6bf [Tathagata Das] Fix serialization issues
f8b7680 [Tathagata Das] Updated doc
33fe43a [Tathagata Das] Added more tests
7997138 [Tathagata Das] Fix style errors
a806710 [Tathagata Das] Fixed unit test and use KinesisInputDStream
40a1709 [Tathagata Das] Fixed KinesisReceiverSuite tests
7e44df6 [Tathagata Das] Added documentation and fixed checkpointing
096383f [Tathagata Das] Added test, and addressed some of the comments.
84a7892 [Tathagata Das] fixed scala style issue
e19e37d [Tathagata Das] Added license
1cd7b66 [Tathagata Das] Updated kinesis receiver
  • Loading branch information
tdas committed Aug 5, 2015
1 parent 781c8d7 commit c2a71f0
Show file tree
Hide file tree
Showing 11 changed files with 605 additions and 197 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -37,16 +37,18 @@ case class SequenceNumberRange(

/** Class representing an array of Kinesis sequence number ranges */
private[kinesis]
case class SequenceNumberRanges(ranges: Array[SequenceNumberRange]) {
case class SequenceNumberRanges(ranges: Seq[SequenceNumberRange]) {
def isEmpty(): Boolean = ranges.isEmpty

def nonEmpty(): Boolean = ranges.nonEmpty

override def toString(): String = ranges.mkString("SequenceNumberRanges(", ", ", ")")
}

private[kinesis]
object SequenceNumberRanges {
def apply(range: SequenceNumberRange): SequenceNumberRanges = {
new SequenceNumberRanges(Array(range))
new SequenceNumberRanges(Seq(range))
}
}

Expand All @@ -66,14 +68,14 @@ class KinesisBackedBlockRDDPartition(
*/
private[kinesis]
class KinesisBackedBlockRDD(
sc: SparkContext,
regionId: String,
endpointUrl: String,
@transient sc: SparkContext,
val regionName: String,
val endpointUrl: String,
@transient blockIds: Array[BlockId],
@transient arrayOfseqNumberRanges: Array[SequenceNumberRanges],
@transient val arrayOfseqNumberRanges: Array[SequenceNumberRanges],
@transient isBlockIdValid: Array[Boolean] = Array.empty,
retryTimeoutMs: Int = 10000,
awsCredentialsOption: Option[SerializableAWSCredentials] = None
val retryTimeoutMs: Int = 10000,
val awsCredentialsOption: Option[SerializableAWSCredentials] = None
) extends BlockRDD[Array[Byte]](sc, blockIds) {

require(blockIds.length == arrayOfseqNumberRanges.length,
Expand Down Expand Up @@ -104,7 +106,7 @@ class KinesisBackedBlockRDD(
}
partition.seqNumberRanges.ranges.iterator.flatMap { range =>
new KinesisSequenceRangeIterator(
credenentials, endpointUrl, regionId, range, retryTimeoutMs)
credenentials, endpointUrl, regionName, range, retryTimeoutMs)
}
}
if (partition.isBlockIdValid) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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 com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream

import org.apache.spark.rdd.RDD
import org.apache.spark.storage.{BlockId, StorageLevel}
import org.apache.spark.streaming.dstream.ReceiverInputDStream
import org.apache.spark.streaming.receiver.Receiver
import org.apache.spark.streaming.scheduler.ReceivedBlockInfo
import org.apache.spark.streaming.{Duration, StreamingContext, Time}

private[kinesis] class KinesisInputDStream(
@transient _ssc: StreamingContext,
streamName: String,
endpointUrl: String,
regionName: String,
initialPositionInStream: InitialPositionInStream,
checkpointAppName: String,
checkpointInterval: Duration,
storageLevel: StorageLevel,
awsCredentialsOption: Option[SerializableAWSCredentials]
) extends ReceiverInputDStream[Array[Byte]](_ssc) {

private[streaming]
override def createBlockRDD(time: Time, blockInfos: Seq[ReceivedBlockInfo]): RDD[Array[Byte]] = {

// This returns true even for when blockInfos is empty
val allBlocksHaveRanges = blockInfos.map { _.metadataOption }.forall(_.nonEmpty)

if (allBlocksHaveRanges) {
// Create a KinesisBackedBlockRDD, even when there are no blocks
val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray
val seqNumRanges = blockInfos.map {
_.metadataOption.get.asInstanceOf[SequenceNumberRanges] }.toArray
val isBlockIdValid = blockInfos.map { _.isBlockIdValid() }.toArray
logDebug(s"Creating KinesisBackedBlockRDD for $time with ${seqNumRanges.length} " +
s"seq number ranges: ${seqNumRanges.mkString(", ")} ")
new KinesisBackedBlockRDD(
context.sc, regionName, endpointUrl, blockIds, seqNumRanges,
isBlockIdValid = isBlockIdValid,
retryTimeoutMs = ssc.graph.batchDuration.milliseconds.toInt,
awsCredentialsOption = awsCredentialsOption)
} else {
logWarning("Kinesis sequence number information was not present with some block metadata," +
" it may not be possible to recover from failures")
super.createBlockRDD(time, blockInfos)
}
}

override def getReceiver(): Receiver[Array[Byte]] = {
new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
checkpointAppName, checkpointInterval, storageLevel, awsCredentialsOption)
}
}
Loading

0 comments on commit c2a71f0

Please sign in to comment.