Skip to content

Commit c35c60f

Browse files
lw-linsrowen
authored andcommitted
[SPARK-14028][STREAMING][KINESIS][TESTS] Remove deprecated methods; fix two other warnings
## What changes were proposed in this pull request? - Removed two methods that has been deprecated since 1.4 - Fixed two other compilation warnings ## How was this patch tested? existing test suits Author: proflin <proflin.me@gmail.com> Closes #11850 from lw-lin/streaming-kinesis-deprecates-warnings.
1 parent 761c2d1 commit c35c60f

File tree

4 files changed

+13
-98
lines changed

4 files changed

+13
-98
lines changed

external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala

Lines changed: 0 additions & 86 deletions
Original file line numberDiff line numberDiff line change
@@ -221,51 +221,6 @@ object KinesisUtils {
221221
}
222222
}
223223

224-
/**
225-
* Create an input stream that pulls messages from a Kinesis stream.
226-
* This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
227-
*
228-
* Note:
229-
*
230-
* - The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
231-
* on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
232-
* gets AWS credentials.
233-
* - The region of the `endpointUrl` will be used for DynamoDB and CloudWatch.
234-
* - The Kinesis application name used by the Kinesis Client Library (KCL) will be the app name
235-
* in [[org.apache.spark.SparkConf]].
236-
*
237-
* @param ssc StreamingContext object
238-
* @param streamName Kinesis stream name
239-
* @param endpointUrl Endpoint url of Kinesis service
240-
* (e.g., https://kinesis.us-east-1.amazonaws.com)
241-
* @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
242-
* See the Kinesis Spark Streaming documentation for more
243-
* details on the different types of checkpoints.
244-
* @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
245-
* worker's initial starting position in the stream.
246-
* The values are either the beginning of the stream
247-
* per Kinesis' limit of 24 hours
248-
* (InitialPositionInStream.TRIM_HORIZON) or
249-
* the tip of the stream (InitialPositionInStream.LATEST).
250-
* @param storageLevel Storage level to use for storing the received objects
251-
* StorageLevel.MEMORY_AND_DISK_2 is recommended.
252-
*/
253-
@deprecated("use other forms of createStream", "1.4.0")
254-
def createStream(
255-
ssc: StreamingContext,
256-
streamName: String,
257-
endpointUrl: String,
258-
checkpointInterval: Duration,
259-
initialPositionInStream: InitialPositionInStream,
260-
storageLevel: StorageLevel
261-
): ReceiverInputDStream[Array[Byte]] = {
262-
ssc.withNamedScope("kinesis stream") {
263-
new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl,
264-
getRegionByEndpoint(endpointUrl), initialPositionInStream, ssc.sc.appName,
265-
checkpointInterval, storageLevel, defaultMessageHandler, None)
266-
}
267-
}
268-
269224
/**
270225
* Create an input stream that pulls messages from a Kinesis stream.
271226
* This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
@@ -453,47 +408,6 @@ object KinesisUtils {
453408
defaultMessageHandler(_), awsAccessKeyId, awsSecretKey)
454409
}
455410

456-
/**
457-
* Create an input stream that pulls messages from a Kinesis stream.
458-
* This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
459-
*
460-
* Note:
461-
* - The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
462-
* on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
463-
* gets AWS credentials.
464-
* - The region of the `endpointUrl` will be used for DynamoDB and CloudWatch.
465-
* - The Kinesis application name used by the Kinesis Client Library (KCL) will be the app name in
466-
* [[org.apache.spark.SparkConf]].
467-
*
468-
* @param jssc Java StreamingContext object
469-
* @param streamName Kinesis stream name
470-
* @param endpointUrl Endpoint url of Kinesis service
471-
* (e.g., https://kinesis.us-east-1.amazonaws.com)
472-
* @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
473-
* See the Kinesis Spark Streaming documentation for more
474-
* details on the different types of checkpoints.
475-
* @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
476-
* worker's initial starting position in the stream.
477-
* The values are either the beginning of the stream
478-
* per Kinesis' limit of 24 hours
479-
* (InitialPositionInStream.TRIM_HORIZON) or
480-
* the tip of the stream (InitialPositionInStream.LATEST).
481-
* @param storageLevel Storage level to use for storing the received objects
482-
* StorageLevel.MEMORY_AND_DISK_2 is recommended.
483-
*/
484-
@deprecated("use other forms of createStream", "1.4.0")
485-
def createStream(
486-
jssc: JavaStreamingContext,
487-
streamName: String,
488-
endpointUrl: String,
489-
checkpointInterval: Duration,
490-
initialPositionInStream: InitialPositionInStream,
491-
storageLevel: StorageLevel
492-
): JavaReceiverInputDStream[Array[Byte]] = {
493-
createStream(
494-
jssc.ssc, streamName, endpointUrl, checkpointInterval, initialPositionInStream, storageLevel)
495-
}
496-
497411
private def getRegionByEndpoint(endpointUrl: String): String = {
498412
RegionUtils.getRegionByEndpoint(endpointUrl).getName()
499413
}

external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.streaming.kinesis;
1919

20+
import com.amazonaws.regions.RegionUtils;
2021
import com.amazonaws.services.kinesis.model.Record;
2122
import org.junit.Test;
2223

@@ -34,11 +35,13 @@
3435
public class JavaKinesisStreamSuite extends LocalJavaStreamingContext {
3536
@Test
3637
public void testKinesisStream() {
37-
// Tests the API, does not actually test data receiving
38-
JavaDStream<byte[]> kinesisStream = KinesisUtils.createStream(ssc, "mySparkStream",
39-
"https://kinesis.us-west-2.amazonaws.com", new Duration(2000),
40-
InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2());
38+
String dummyEndpointUrl = KinesisTestUtils.defaultEndpointUrl();
39+
String dummyRegionName = RegionUtils.getRegionByEndpoint(dummyEndpointUrl).getName();
4140

41+
// Tests the API, does not actually test data receiving
42+
JavaDStream<byte[]> kinesisStream = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
43+
dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, new Duration(2000),
44+
StorageLevel.MEMORY_AND_DISK_2());
4245
ssc.stop();
4346
}
4447

external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ trait KinesisFunSuite extends SparkFunSuite {
4040
if (shouldRunTests) {
4141
body
4242
} else {
43-
ignore(s"$message [enable by setting env var $envVarNameForEnablingTests=1]")()
43+
ignore(s"$message [enable by setting env var $envVarNameForEnablingTests=1]")(())
4444
}
4545
}
4646
}

external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -99,14 +99,10 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun
9999
}
100100

101101
test("KinesisUtils API") {
102-
// Tests the API, does not actually test data receiving
103-
val kinesisStream1 = KinesisUtils.createStream(ssc, "mySparkStream",
104-
dummyEndpointUrl, Seconds(2),
105-
InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2)
106-
val kinesisStream2 = KinesisUtils.createStream(ssc, "myAppNam", "mySparkStream",
102+
val kinesisStream1 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
107103
dummyEndpointUrl, dummyRegionName,
108104
InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2)
109-
val kinesisStream3 = KinesisUtils.createStream(ssc, "myAppNam", "mySparkStream",
105+
val kinesisStream2 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
110106
dummyEndpointUrl, dummyRegionName,
111107
InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2,
112108
dummyAWSAccessKey, dummyAWSSecretKey)
@@ -154,7 +150,9 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun
154150

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

160158
// Verify that the KinesisBackedBlockRDD has isBlockValid = false when blocks are invalid

0 commit comments

Comments
 (0)