-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-9216][Streaming] Define KinesisBackedBlockRDDs #7578
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
3ae0814
Added KinesisBackedBlockRDD
tdas 528e206
Merge remote-tracking branch 'apache-github/master' into kinesis-rdd
tdas 5da3995
Changed KinesisSuiteHelper to KinesisFunSuite
tdas 4a36096
Add license
tdas 575bdbc
Fix scala style issues
tdas 8874b70
Updated Kinesis RDD
tdas f6e35c8
Added retry logic to make it more robust
tdas d3d64d1
Minor update
tdas c4f25d2
Addressed comment
tdas 3f40c2d
Addressed comments
tdas 5082a30
Fixed scala style
tdas 543d208
Fixed scala style
tdas File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
285 changes: 285 additions & 0 deletions
285
...kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,285 @@ | ||
| /* | ||
| * 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.JavaConversions._ | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import com.amazonaws.auth.{AWSCredentials, DefaultAWSCredentialsProviderChain} | ||
| import com.amazonaws.services.kinesis.AmazonKinesisClient | ||
| import com.amazonaws.services.kinesis.model._ | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.rdd.{BlockRDD, BlockRDDPartition} | ||
| import org.apache.spark.storage.BlockId | ||
| import org.apache.spark.util.NextIterator | ||
|
|
||
|
|
||
| /** Class representing a range of Kinesis sequence numbers. Both sequence numbers are inclusive. */ | ||
| private[kinesis] | ||
| case class SequenceNumberRange( | ||
| streamName: String, shardId: String, fromSeqNumber: String, toSeqNumber: String) | ||
|
|
||
| /** Class representing an array of Kinesis sequence number ranges */ | ||
| private[kinesis] | ||
| case class SequenceNumberRanges(ranges: Array[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)) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| /** Partition storing the information of the ranges of Kinesis sequence numbers to read */ | ||
| private[kinesis] | ||
| class KinesisBackedBlockRDDPartition( | ||
| idx: Int, | ||
| blockId: BlockId, | ||
| val isBlockIdValid: Boolean, | ||
| val seqNumberRanges: SequenceNumberRanges | ||
| ) extends BlockRDDPartition(blockId, idx) | ||
|
|
||
| /** | ||
| * A BlockRDD where the block data is backed by Kinesis, which can accessed using the | ||
| * sequence numbers of the corresponding blocks. | ||
| */ | ||
| private[kinesis] | ||
| class KinesisBackedBlockRDD( | ||
| sc: SparkContext, | ||
| regionId: String, | ||
| endpointUrl: String, | ||
| @transient blockIds: Array[BlockId], | ||
| @transient arrayOfseqNumberRanges: Array[SequenceNumberRanges], | ||
| @transient isBlockIdValid: Array[Boolean] = Array.empty, | ||
| retryTimeoutMs: Int = 10000, | ||
| awsCredentialsOption: Option[SerializableAWSCredentials] = None | ||
| ) extends BlockRDD[Array[Byte]](sc, blockIds) { | ||
|
|
||
| require(blockIds.length == arrayOfseqNumberRanges.length, | ||
| "Number of blockIds is not equal to the number of sequence number ranges") | ||
|
|
||
| override def isValid(): Boolean = true | ||
|
|
||
| override def getPartitions: Array[Partition] = { | ||
| Array.tabulate(blockIds.length) { i => | ||
| val isValid = if (isBlockIdValid.length == 0) true else isBlockIdValid(i) | ||
| new KinesisBackedBlockRDDPartition(i, blockIds(i), isValid, arrayOfseqNumberRanges(i)) | ||
| } | ||
| } | ||
|
|
||
| override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { | ||
| val blockManager = SparkEnv.get.blockManager | ||
| val partition = split.asInstanceOf[KinesisBackedBlockRDDPartition] | ||
| val blockId = partition.blockId | ||
|
|
||
| def getBlockFromBlockManager(): Option[Iterator[Array[Byte]]] = { | ||
| logDebug(s"Read partition data of $this from block manager, block $blockId") | ||
| blockManager.get(blockId).map(_.data.asInstanceOf[Iterator[Array[Byte]]]) | ||
| } | ||
|
|
||
| def getBlockFromKinesis(): Iterator[Array[Byte]] = { | ||
| val credenentials = awsCredentialsOption.getOrElse { | ||
| new DefaultAWSCredentialsProviderChain().getCredentials() | ||
| } | ||
| partition.seqNumberRanges.ranges.iterator.flatMap { range => | ||
| new KinesisSequenceRangeIterator( | ||
| credenentials, endpointUrl, regionId, range, retryTimeoutMs) | ||
| } | ||
| } | ||
| if (partition.isBlockIdValid) { | ||
| getBlockFromBlockManager().getOrElse { getBlockFromKinesis() } | ||
| } else { | ||
| getBlockFromKinesis() | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * An iterator that return the Kinesis data based on the given range of sequence numbers. | ||
| * Internally, it repeatedly fetches sets of records starting from the fromSequenceNumber, | ||
| * until the endSequenceNumber is reached. | ||
| */ | ||
| private[kinesis] | ||
| class KinesisSequenceRangeIterator( | ||
| credentials: AWSCredentials, | ||
| endpointUrl: String, | ||
| regionId: String, | ||
| range: SequenceNumberRange, | ||
| retryTimeoutMs: Int | ||
| ) extends NextIterator[Array[Byte]] with Logging { | ||
|
|
||
| private val client = new AmazonKinesisClient(credentials) | ||
| private val streamName = range.streamName | ||
| private val shardId = range.shardId | ||
|
|
||
| private var toSeqNumberReceived = false | ||
| private var lastSeqNumber: String = null | ||
| private var internalIterator: Iterator[Record] = null | ||
|
|
||
| client.setEndpoint(endpointUrl, "kinesis", regionId) | ||
|
|
||
| override protected def getNext(): Array[Byte] = { | ||
| var nextBytes: Array[Byte] = null | ||
| if (toSeqNumberReceived) { | ||
| finished = true | ||
| } else { | ||
|
|
||
| if (internalIterator == null) { | ||
|
|
||
| // If the internal iterator has not been initialized, | ||
| // then fetch records from starting sequence number | ||
| internalIterator = getRecords(ShardIteratorType.AT_SEQUENCE_NUMBER, range.fromSeqNumber) | ||
| } else if (!internalIterator.hasNext) { | ||
|
|
||
| // If the internal iterator does not have any more records, | ||
| // then fetch more records after the last consumed sequence number | ||
| internalIterator = getRecords(ShardIteratorType.AFTER_SEQUENCE_NUMBER, lastSeqNumber) | ||
| } | ||
|
|
||
| if (!internalIterator.hasNext) { | ||
|
|
||
| // If the internal iterator still does not have any data, then throw exception | ||
| // and terminate this iterator | ||
| finished = true | ||
| throw new SparkException( | ||
| s"Could not read until the end sequence number of the range: $range") | ||
| } else { | ||
|
|
||
| // Get the record, copy the data into a byte array and remember its sequence number | ||
| val nextRecord: Record = internalIterator.next() | ||
| val byteBuffer = nextRecord.getData() | ||
| nextBytes = new Array[Byte](byteBuffer.remaining()) | ||
| byteBuffer.get(nextBytes) | ||
| lastSeqNumber = nextRecord.getSequenceNumber() | ||
|
|
||
| // If the this record's sequence number matches the stopping sequence number, then make sure | ||
| // the iterator is marked finished next time getNext() is called | ||
| if (nextRecord.getSequenceNumber == range.toSeqNumber) { | ||
| toSeqNumberReceived = true | ||
| } | ||
| } | ||
|
|
||
| } | ||
| nextBytes | ||
| } | ||
|
|
||
| override protected def close(): Unit = { | ||
| client.shutdown() | ||
| } | ||
|
|
||
| /** | ||
| * Get records starting from or after the given sequence number. | ||
| */ | ||
| private def getRecords(iteratorType: ShardIteratorType, seqNum: String): Iterator[Record] = { | ||
| val shardIterator = getKinesisIterator(iteratorType, seqNum) | ||
| val result = getRecordsAndNextKinesisIterator(shardIterator) | ||
| result._1 | ||
| } | ||
|
|
||
| /** | ||
| * Get the records starting from using a Kinesis shard iterator (which is a progress handle | ||
| * to get records from Kinesis), and get the next shard iterator for next consumption. | ||
| */ | ||
| private def getRecordsAndNextKinesisIterator( | ||
| shardIterator: String): (Iterator[Record], String) = { | ||
| val getRecordsRequest = new GetRecordsRequest | ||
| getRecordsRequest.setRequestCredentials(credentials) | ||
| getRecordsRequest.setShardIterator(shardIterator) | ||
| val getRecordsResult = retryOrTimeout[GetRecordsResult]( | ||
| s"getting records using shard iterator") { | ||
| client.getRecords(getRecordsRequest) | ||
| } | ||
| (getRecordsResult.getRecords.iterator(), getRecordsResult.getNextShardIterator) | ||
| } | ||
|
|
||
| /** | ||
| * Get the Kinesis shard iterator for getting records starting from or after the given | ||
| * sequence number. | ||
| */ | ||
| private def getKinesisIterator( | ||
| iteratorType: ShardIteratorType, | ||
| sequenceNumber: String): String = { | ||
| val getShardIteratorRequest = new GetShardIteratorRequest | ||
| getShardIteratorRequest.setRequestCredentials(credentials) | ||
| getShardIteratorRequest.setStreamName(streamName) | ||
| getShardIteratorRequest.setShardId(shardId) | ||
| getShardIteratorRequest.setShardIteratorType(iteratorType.toString) | ||
| getShardIteratorRequest.setStartingSequenceNumber(sequenceNumber) | ||
| val getShardIteratorResult = retryOrTimeout[GetShardIteratorResult]( | ||
| s"getting shard iterator from sequence number $sequenceNumber") { | ||
| client.getShardIterator(getShardIteratorRequest) | ||
| } | ||
| getShardIteratorResult.getShardIterator | ||
| } | ||
|
|
||
| /** Helper method to retry Kinesis API request with exponential backoff and timeouts */ | ||
| private def retryOrTimeout[T](message: String)(body: => T): T = { | ||
| import KinesisSequenceRangeIterator._ | ||
|
|
||
| var startTimeMs = System.currentTimeMillis() | ||
| var retryCount = 0 | ||
| var waitTimeMs = MIN_RETRY_WAIT_TIME_MS | ||
| var result: Option[T] = None | ||
| var lastError: Throwable = null | ||
|
|
||
| def isTimedOut = (System.currentTimeMillis() - startTimeMs) >= retryTimeoutMs | ||
| def isMaxRetryDone = retryCount >= MAX_RETRIES | ||
|
|
||
| while (result.isEmpty && !isTimedOut && !isMaxRetryDone) { | ||
| if (retryCount > 0) { // wait only if this is a retry | ||
| Thread.sleep(waitTimeMs) | ||
| waitTimeMs *= 2 // if you have waited, then double wait time for next round | ||
| } | ||
| try { | ||
| result = Some(body) | ||
| } catch { | ||
| case NonFatal(t) => | ||
| lastError = t | ||
| t match { | ||
| case ptee: ProvisionedThroughputExceededException => | ||
| logWarning(s"Error while $message [attempt = ${retryCount + 1}]", ptee) | ||
| case e: Throwable => | ||
| throw new SparkException(s"Error while $message", e) | ||
| } | ||
| } | ||
| retryCount += 1 | ||
| } | ||
| result.getOrElse { | ||
| if (isTimedOut) { | ||
| throw new SparkException( | ||
| s"Timed out after $retryTimeoutMs ms while $message, last exception: ", lastError) | ||
| } else { | ||
| throw new SparkException( | ||
| s"Gave up after $retryCount retries while $message, last exception: ", lastError) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[streaming] | ||
| object KinesisSequenceRangeIterator { | ||
| val MAX_RETRIES = 3 | ||
| val MIN_RETRY_WAIT_TIME_MS = 100 | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
toSeqNumberis inclusive, right? Could you document it? I think it's not intuitive.