forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 3
Add Kinesis Source #31
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
Merged
tdas
merged 16 commits into
tdas:streaming-df-kinesis
from
zsxwing:streaming-df-kinesis
Feb 24, 2016
Merged
Changes from all commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
92dced8
Clean up blocks for Kinesis source; add unit tests
zsxwing ed8bdc3
Add Kinesis DefaultSource
zsxwing a0c03e6
Revert changes to core
zsxwing 6d39214
Hack StreamBlockId
zsxwing 8cdb269
Address TD's comments
zsxwing e26eee9
Move KinesisDataFetcher to a new file
zsxwing 6a01b28
Add stress test and fix a bug in compareTo
zsxwing 435201e
Better option error messages and unit tests; address other comments
zsxwing 3629cd9
Address more
zsxwing d3e7887
Fix the block id and generate block id in the driver
zsxwing e058a64
Refactor
zsxwing aef4027
Add abstract
zsxwing f85608f
Downgrade KCL to 1.4.0
zsxwing 2ae6285
Address comments for tests
zsxwing b55260e
Fix indent
zsxwing 8ac76ca
Fix indent
zsxwing 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
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
115 changes: 115 additions & 0 deletions
115
extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/DefaultSource.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,115 @@ | ||
| /* | ||
| * 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.AmazonClientException | ||
| import com.amazonaws.auth.DefaultAWSCredentialsProviderChain | ||
| import com.amazonaws.regions.RegionUtils | ||
| import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream | ||
|
|
||
| import org.apache.spark.sql.SQLContext | ||
| import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap | ||
| import org.apache.spark.sql.execution.streaming.Source | ||
| import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| class DefaultSource extends StreamSourceProvider with DataSourceRegister { | ||
|
|
||
| override def shortName(): String = "kinesis" | ||
|
|
||
| override def createSource( | ||
| sqlContext: SQLContext, | ||
| schema: Option[StructType], | ||
| providerName: String, | ||
| parameters: Map[String, String]): Source = { | ||
| val caseInsensitiveOptions = new CaseInsensitiveMap(parameters) | ||
|
|
||
| val streams = caseInsensitiveOptions.getOrElse("stream", { | ||
| throw new IllegalArgumentException( | ||
| "Option 'stream' must be specified. Examples: " + | ||
| """option("stream", "stream1"), option("stream", "stream1,stream2")""") | ||
| }).split(",", -1).toSet | ||
|
|
||
| if (streams.isEmpty || streams.exists(_.isEmpty)) { | ||
| throw new IllegalArgumentException( | ||
| "Option 'stream' is invalid, as stream names cannot be empty.") | ||
| } | ||
|
|
||
| val regionOption = caseInsensitiveOptions.get("region") | ||
| val endpointOption = caseInsensitiveOptions.get("endpoint") | ||
| val (region, endpoint) = (regionOption, endpointOption) match { | ||
| case (Some(_region), Some(_endpoint)) => | ||
| if (RegionUtils.getRegionByEndpoint(_endpoint).getName != _region) { | ||
| throw new IllegalArgumentException( | ||
| s"'region'(${_region}) doesn't match to 'endpoint'(${_endpoint})") | ||
| } | ||
| (_region, _endpoint) | ||
| case (Some(_region), None) => | ||
| (_region, RegionUtils.getRegion(_region).getServiceEndpoint("kinesis")) | ||
| case (None, Some(_endpoint)) => | ||
| (RegionUtils.getRegionByEndpoint(_endpoint).getName, _endpoint) | ||
| case (None, None) => | ||
| throw new IllegalArgumentException( | ||
| "Either option 'region' or option 'endpoint' must be specified. Examples: " + | ||
| """option("region", "us-west-2"), """ + | ||
| """option("endpoint", "https://kinesis.us-west-2.amazonaws.com")""") | ||
| } | ||
|
|
||
| val initialPosInStream = | ||
| caseInsensitiveOptions.getOrElse("position", InitialPositionInStream.LATEST.name) match { | ||
| case pos if pos.toUpperCase == InitialPositionInStream.LATEST.name => | ||
| InitialPositionInStream.LATEST | ||
| case pos if pos.toUpperCase == InitialPositionInStream.TRIM_HORIZON.name => | ||
| InitialPositionInStream.TRIM_HORIZON | ||
| case pos => | ||
| throw new IllegalArgumentException(s"Unknown value of option 'position': $pos") | ||
| } | ||
|
|
||
| val accessKeyOption = caseInsensitiveOptions.get("accessKey") | ||
| val secretKeyOption = caseInsensitiveOptions.get("secretKey") | ||
| val credentials = (accessKeyOption, secretKeyOption) match { | ||
| case (Some(accessKey), Some(secretKey)) => | ||
| new SerializableAWSCredentials(accessKey, secretKey) | ||
| case (Some(accessKey), None) => | ||
| throw new IllegalArgumentException( | ||
| s"'accessKey' is set but 'secretKey' is not found") | ||
| case (None, Some(secretKey)) => | ||
| throw new IllegalArgumentException( | ||
| s"'secretKey' is set but 'accessKey' is not found") | ||
| case (None, None) => | ||
| try { | ||
| SerializableAWSCredentials(new DefaultAWSCredentialsProviderChain().getCredentials()) | ||
| } catch { | ||
| case _: AmazonClientException => | ||
| throw new IllegalArgumentException( | ||
| "No credential found using default AWS provider chain. Specify credentials using " + | ||
| "options 'accessKey' and 'secretKey'. Examples: " + | ||
| """option("accessKey", "your-aws-access-key"), """ + | ||
| """option("secretKey", "your-aws-secret-key")""") | ||
| } | ||
| } | ||
|
|
||
| new KinesisSource( | ||
| sqlContext, | ||
| region, | ||
| endpoint, | ||
| streams, | ||
| initialPosInStream, | ||
| credentials) | ||
| } | ||
| } | ||
203 changes: 203 additions & 0 deletions
203
...as/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDataFetcher.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,203 @@ | ||
| /* | ||
| * 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.JavaConverters._ | ||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import com.amazonaws.services.kinesis.AmazonKinesisClient | ||
| import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream | ||
| import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord | ||
| import com.amazonaws.services.kinesis.model._ | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.network.util.JavaUtils | ||
| import org.apache.spark.storage.{BlockId, StorageLevel} | ||
|
|
||
| /** | ||
| * However, this class runs in the driver so could be a bottleneck. | ||
| */ | ||
| private[kinesis] class KinesisDataFetcher( | ||
| credentials: SerializableAWSCredentials, | ||
| endpointUrl: String, | ||
| fromSeqNums: Seq[(Shard, Option[String], BlockId)], | ||
| initialPositionInStream: InitialPositionInStream, | ||
| readTimeoutMs: Long = 2000L) extends Serializable with Logging { | ||
|
|
||
| /** | ||
| * Use lazy because the client needs to be created in executors | ||
| */ | ||
| @transient private lazy val client = new AmazonKinesisClient(credentials) | ||
|
|
||
| /** | ||
| * Launch a Spark job to fetch latest data from the specified `shard`s. This method will try to | ||
| * fetch arriving data in `readTimeoutMs` milliseconds so as to get the latest sequence numbers. | ||
| * New data will be pushed to the block manager to avoid fetching them again. | ||
| * | ||
| * This is a workaround since Kinesis doesn't provider an API to fetch the latest sequence number. | ||
| */ | ||
| def fetch(sc: SparkContext): Array[(BlockId, SequenceNumberRange)] = { | ||
|
Owner
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add scala docs. |
||
| sc.makeRDD(fromSeqNums, fromSeqNums.size).map { | ||
| case (shard, fromSeqNum, blockId) => fetchPartition(shard, fromSeqNum, blockId) | ||
| }.collect().flatten | ||
| } | ||
|
|
||
| /** | ||
| * Fetch latest data from the specified `shard` since `fromSeqNum`. This method will try to fetch | ||
| * arriving data in `readTimeoutMs` milliseconds so as to get the latest sequence number. New data | ||
| * will be pushed to the block manager to avoid fetching them again. | ||
| * | ||
| * This is a workaround since Kinesis doesn't provider an API to fetch the latest sequence number. | ||
| */ | ||
| private def fetchPartition( | ||
|
Owner
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add scala docs |
||
| shard: Shard, | ||
| fromSeqNum: Option[String], | ||
| blockId: BlockId): Option[(BlockId, SequenceNumberRange)] = { | ||
| client.setEndpoint(endpointUrl) | ||
|
|
||
| val endTime = System.currentTimeMillis + readTimeoutMs | ||
| def timeLeft = math.max(endTime - System.currentTimeMillis, 0) | ||
|
|
||
| val buffer = new ArrayBuffer[Array[Byte]] | ||
| var firstSeqNumber: String = null | ||
| var lastSeqNumber: String = fromSeqNum.orNull | ||
| var lastIterator: String = null | ||
| try { | ||
| logDebug(s"Trying to fetch data from $shard, from seq num $lastSeqNumber") | ||
|
|
||
| while (timeLeft > 0) { | ||
| val (records, nextIterator) = retryOrTimeout("getting shard iterator", timeLeft) { | ||
| if (lastIterator == null) { | ||
| lastIterator = if (lastSeqNumber != null) { | ||
| getKinesisIterator(shard, ShardIteratorType.AFTER_SEQUENCE_NUMBER, lastSeqNumber) | ||
| } else { | ||
| if (initialPositionInStream == InitialPositionInStream.LATEST) { | ||
| getKinesisIterator(shard, ShardIteratorType.LATEST, lastSeqNumber) | ||
| } else { | ||
| getKinesisIterator(shard, ShardIteratorType.TRIM_HORIZON, lastSeqNumber) | ||
| } | ||
| } | ||
| } | ||
| getRecordsAndNextKinesisIterator(lastIterator) | ||
| } | ||
|
|
||
| records.foreach { record => | ||
| buffer += JavaUtils.bufferToArray(record.getData()) | ||
| if (firstSeqNumber == null) { | ||
| firstSeqNumber = record.getSequenceNumber | ||
| } | ||
| lastSeqNumber = record.getSequenceNumber | ||
| } | ||
|
|
||
| lastIterator = nextIterator | ||
| } | ||
|
|
||
| if (buffer.nonEmpty) { | ||
| SparkEnv.get.blockManager.putIterator(blockId, buffer.iterator, StorageLevel.MEMORY_ONLY) | ||
| val range = SequenceNumberRange( | ||
| shard.streamName, shard.shardId, firstSeqNumber, lastSeqNumber) | ||
| logDebug(s"Received block $blockId having range $range from shard $shard") | ||
| Some(blockId -> range) | ||
| } else { | ||
| None | ||
| } | ||
| } catch { | ||
| case NonFatal(e) => | ||
| logWarning(s"Error fetching data from shard $shard", e) | ||
| None | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * 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): (Seq[Record], String) = { | ||
| val getRecordsRequest = new GetRecordsRequest().withShardIterator(shardIterator) | ||
| getRecordsRequest.setRequestCredentials(credentials) | ||
| val getRecordsResult = client.getRecords(getRecordsRequest) | ||
| // De-aggregate records, if KPL was used in producing the records. The KCL automatically | ||
| // handles de-aggregation during regular operation. This code path is used during recovery | ||
| val records = UserRecord.deaggregate(getRecordsResult.getRecords) | ||
| logTrace( | ||
| s"Got ${records.size()} records and next iterator ${getRecordsResult.getNextShardIterator}") | ||
| (records.asScala, getRecordsResult.getNextShardIterator) | ||
| } | ||
|
|
||
| /** | ||
| * Get the Kinesis shard iterator for getting records starting from or after the given | ||
| * sequence number. | ||
| */ | ||
| private def getKinesisIterator( | ||
| shard: Shard, | ||
| iteratorType: ShardIteratorType, | ||
| sequenceNumber: String): String = { | ||
| val getShardIteratorRequest = new GetShardIteratorRequest() | ||
| .withStreamName(shard.streamName) | ||
| .withShardId(shard.shardId) | ||
| .withShardIteratorType(iteratorType.toString) | ||
| .withStartingSequenceNumber(sequenceNumber) | ||
| getShardIteratorRequest.setRequestCredentials(credentials) | ||
| val getShardIteratorResult = client.getShardIterator(getShardIteratorRequest) | ||
| logTrace(s"Shard $shard: Got iterator ${getShardIteratorResult.getShardIterator}") | ||
| getShardIteratorResult.getShardIterator | ||
| } | ||
|
|
||
| /** Helper method to retry Kinesis API request with exponential backoff and timeouts */ | ||
| private def retryOrTimeout[T](message: String, retryTimeoutMs: Long)(body: => T): T = { | ||
| import KinesisSequenceRangeIterator._ | ||
| val 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) | ||
| } | ||
| } | ||
| } | ||
| } | ||
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.
Check whether access keys are available (either specified, or loaded from a provider) right here. Otherwise, it fails later, and gives a bad error message like
So better to resolve the credentials right here and pass it on to the source without an option.