-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-20189][DStream] Fix spark kinesis testcases to remove deprecated createStream and use Builders #17506
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,7 +22,6 @@ 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} | ||
|
|
@@ -173,11 +172,15 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun | |
| * 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 stream = KinesisInputDStream.builder.streamingContext(ssc) | ||
| .checkpointAppName(appName) | ||
| .streamName(testUtils.streamName) | ||
| .endpointUrl(testUtils.endpointUrl) | ||
| .regionName(testUtils.regionName) | ||
| .initialPositionInStream(InitialPositionInStream.LATEST) | ||
| .checkpointInterval(Seconds(10)) | ||
| .storageLevel(StorageLevel.MEMORY_ONLY) | ||
| .build() | ||
|
|
||
| val collected = new mutable.HashSet[Int] | ||
| stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd => | ||
|
|
@@ -198,12 +201,17 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun | |
| } | ||
|
|
||
| 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) | ||
|
|
||
| val stream = KinesisInputDStream.builder.streamingContext(ssc) | ||
| .checkpointAppName(appName) | ||
| .streamName(testUtils.streamName) | ||
| .endpointUrl(testUtils.endpointUrl) | ||
| .regionName(testUtils.regionName) | ||
| .initialPositionInStream(InitialPositionInStream.LATEST) | ||
| .checkpointInterval(Seconds(10)) | ||
| .storageLevel(StorageLevel.MEMORY_ONLY) | ||
| .buildWithMessageHandler(addFive(_)) | ||
|
|
||
| stream shouldBe a [ReceiverInputDStream[_]] | ||
|
|
||
|
|
@@ -233,11 +241,15 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun | |
| 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 stream = KinesisInputDStream.builder.streamingContext(ssc) | ||
|
Member
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. Just for my understanding, it's no longer necessary to pass in the credentials explicitly?
Author
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. Yes thats true. The KinesisInputDStream.builder uses the Dafault credentials if no creds are passed, and the default creds work with both AWS Key and Session tokens. |
||
| .checkpointAppName(localAppName) | ||
| .streamName(localTestUtils.streamName) | ||
| .endpointUrl(localTestUtils.endpointUrl) | ||
| .regionName(localTestUtils.regionName) | ||
| .initialPositionInStream(InitialPositionInStream.LATEST) | ||
| .checkpointInterval(Seconds(10)) | ||
| .storageLevel(StorageLevel.MEMORY_ONLY) | ||
| .build() | ||
|
|
||
| val collected = new mutable.HashSet[Int] | ||
| stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd => | ||
|
|
@@ -303,13 +315,17 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun | |
| 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) | ||
| val kinesisStream = KinesisInputDStream.builder.streamingContext(ssc) | ||
| .checkpointAppName(appName) | ||
| .streamName(testUtils.streamName) | ||
| .endpointUrl(testUtils.endpointUrl) | ||
| .regionName(testUtils.regionName) | ||
| .initialPositionInStream(InitialPositionInStream.LATEST) | ||
| .checkpointInterval(Seconds(10)) | ||
| .storageLevel(StorageLevel.MEMORY_ONLY) | ||
| .build() | ||
|
|
||
| // Verify that the generated RDDs are KinesisBackedBlockRDDs, and collect the data in each batch | ||
| kinesisStream.foreachRDD((rdd: RDD[Array[Byte]], time: Time) => { | ||
|
|
||
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.
Is this related?
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.
Yes this is required.
KinesisUtils used to send a cleaned Handler while creating stream, so we do this in the KinesisInputDstream now.
val cleanedHandler = ssc.sc.clean(messageHandler)