aboutsummaryrefslogtreecommitdiff
path: root/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala
blob: 1298463bfba1e6a80636935fc3507702bb81dba0 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
/*
 * 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.reflect.ClassTag

import com.amazonaws.regions.RegionUtils
import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
import com.amazonaws.services.kinesis.model.Record

import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.{Duration, StreamingContext}
import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext}
import org.apache.spark.streaming.dstream.ReceiverInputDStream

object KinesisUtils {
  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param ssc StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param messageHandler A custom message handler that can generate a generic output from a
   *                       Kinesis `Record`, which contains both message data, and metadata.
   *
   * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
   * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
   * gets the AWS credentials.
   */
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream[T: ClassTag](
      ssc: StreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      messageHandler: Record => T): ReceiverInputDStream[T] = {
    val cleanedHandler = ssc.sc.clean(messageHandler)
    // Setting scope to override receiver stream's scope of "receiver stream"
    ssc.withNamedScope("kinesis stream") {
      new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName),
        initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel,
        cleanedHandler, DefaultCredentials, None, None)
    }
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param ssc StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param messageHandler A custom message handler that can generate a generic output from a
   *                       Kinesis `Record`, which contains both message data, and metadata.
   * @param awsAccessKeyId  AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
   * @param awsSecretKey  AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
   *
   * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
   * is enabled. Make sure that your checkpoint directory is secure.
   */
  // scalastyle:off
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream[T: ClassTag](
      ssc: StreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      messageHandler: Record => T,
      awsAccessKeyId: String,
      awsSecretKey: String): ReceiverInputDStream[T] = {
    // scalastyle:on
    val cleanedHandler = ssc.sc.clean(messageHandler)
    ssc.withNamedScope("kinesis stream") {
      val kinesisCredsProvider = BasicCredentials(
        awsAccessKeyId = awsAccessKeyId,
        awsSecretKey = awsSecretKey)
      new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName),
        initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel,
        cleanedHandler, kinesisCredsProvider, None, None)
    }
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param ssc StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param messageHandler A custom message handler that can generate a generic output from a
   *                       Kinesis `Record`, which contains both message data, and metadata.
   * @param awsAccessKeyId  AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
   * @param awsSecretKey  AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
   * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from
   *                         Kinesis stream.
   * @param stsSessionName Name to uniquely identify STS sessions if multiple princples assume
   *                       the same role.
   * @param stsExternalId External ID that can be used to validate against the assumed IAM role's
   *                      trust policy.
   *
   * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
   * is enabled. Make sure that your checkpoint directory is secure.
   */
  // scalastyle:off
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream[T: ClassTag](
      ssc: StreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      messageHandler: Record => T,
      awsAccessKeyId: String,
      awsSecretKey: String,
      stsAssumeRoleArn: String,
      stsSessionName: String,
      stsExternalId: String): ReceiverInputDStream[T] = {
    // scalastyle:on
    val cleanedHandler = ssc.sc.clean(messageHandler)
    ssc.withNamedScope("kinesis stream") {
      val kinesisCredsProvider = STSCredentials(
        stsRoleArn = stsAssumeRoleArn,
        stsSessionName = stsSessionName,
        stsExternalId = Option(stsExternalId),
        longLivedCreds = BasicCredentials(
          awsAccessKeyId = awsAccessKeyId,
          awsSecretKey = awsSecretKey))
      new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName),
        initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel,
        cleanedHandler, kinesisCredsProvider, None, None)
    }
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param ssc StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   *
   * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
   * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
   * gets the AWS credentials.
   */
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream(
      ssc: StreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel): ReceiverInputDStream[Array[Byte]] = {
    // Setting scope to override receiver stream's scope of "receiver stream"
    ssc.withNamedScope("kinesis stream") {
      new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName),
        initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel,
        KinesisInputDStream.defaultMessageHandler, DefaultCredentials, None, None)
    }
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param ssc StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param awsAccessKeyId  AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
   * @param awsSecretKey  AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
   *
   * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
   * is enabled. Make sure that your checkpoint directory is secure.
   */
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream(
      ssc: StreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      awsAccessKeyId: String,
      awsSecretKey: String): ReceiverInputDStream[Array[Byte]] = {
    ssc.withNamedScope("kinesis stream") {
      val kinesisCredsProvider = BasicCredentials(
        awsAccessKeyId = awsAccessKeyId,
        awsSecretKey = awsSecretKey)
      new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName),
        initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel,
        KinesisInputDStream.defaultMessageHandler, kinesisCredsProvider, None, None)
    }
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param jssc Java StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param messageHandler A custom message handler that can generate a generic output from a
   *                       Kinesis `Record`, which contains both message data, and metadata.
   * @param recordClass Class of the records in DStream
   *
   * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
   * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
   * gets the AWS credentials.
   */
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream[T](
      jssc: JavaStreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      messageHandler: JFunction[Record, T],
      recordClass: Class[T]): JavaReceiverInputDStream[T] = {
    implicit val recordCmt: ClassTag[T] = ClassTag(recordClass)
    val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_))
    createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
      initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler)
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param jssc Java StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param messageHandler A custom message handler that can generate a generic output from a
   *                       Kinesis `Record`, which contains both message data, and metadata.
   * @param recordClass Class of the records in DStream
   * @param awsAccessKeyId  AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
   * @param awsSecretKey  AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
   *
   * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
   * is enabled. Make sure that your checkpoint directory is secure.
   */
  // scalastyle:off
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream[T](
      jssc: JavaStreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      messageHandler: JFunction[Record, T],
      recordClass: Class[T],
      awsAccessKeyId: String,
      awsSecretKey: String): JavaReceiverInputDStream[T] = {
    // scalastyle:on
    implicit val recordCmt: ClassTag[T] = ClassTag(recordClass)
    val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_))
    createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
      initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler,
      awsAccessKeyId, awsSecretKey)
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param jssc Java StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param messageHandler A custom message handler that can generate a generic output from a
   *                       Kinesis `Record`, which contains both message data, and metadata.
   * @param recordClass Class of the records in DStream
   * @param awsAccessKeyId  AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
   * @param awsSecretKey  AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
   * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from
   *                         Kinesis stream.
   * @param stsSessionName Name to uniquely identify STS sessions if multiple princples assume
   *                       the same role.
   * @param stsExternalId External ID that can be used to validate against the assumed IAM role's
   *                      trust policy.
   *
   * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
   * is enabled. Make sure that your checkpoint directory is secure.
   */
  // scalastyle:off
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream[T](
      jssc: JavaStreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      messageHandler: JFunction[Record, T],
      recordClass: Class[T],
      awsAccessKeyId: String,
      awsSecretKey: String,
      stsAssumeRoleArn: String,
      stsSessionName: String,
      stsExternalId: String): JavaReceiverInputDStream[T] = {
    // scalastyle:on
    implicit val recordCmt: ClassTag[T] = ClassTag(recordClass)
    val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_))
    createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
      initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler,
      awsAccessKeyId, awsSecretKey, stsAssumeRoleArn, stsSessionName, stsExternalId)
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param jssc Java StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   *
   * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
   * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
   * gets the AWS credentials.
   */
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream(
      jssc: JavaStreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel
    ): JavaReceiverInputDStream[Array[Byte]] = {
    createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
      initialPositionInStream, checkpointInterval, storageLevel,
      KinesisInputDStream.defaultMessageHandler(_))
  }

  /**
   * Create an input stream that pulls messages from a Kinesis stream.
   * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
   *
   * @param jssc Java StreamingContext object
   * @param kinesisAppName  Kinesis application name used by the Kinesis Client Library
   *                        (KCL) to update DynamoDB
   * @param streamName   Kinesis stream name
   * @param endpointUrl  Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
   * @param regionName   Name of region used by the Kinesis Client Library (KCL) to update
   *                     DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
   * @param initialPositionInStream  In the absence of Kinesis checkpoint info, this is the
   *                                 worker's initial starting position in the stream.
   *                                 The values are either the beginning of the stream
   *                                 per Kinesis' limit of 24 hours
   *                                 (InitialPositionInStream.TRIM_HORIZON) or
   *                                 the tip of the stream (InitialPositionInStream.LATEST).
   * @param checkpointInterval  Checkpoint interval for Kinesis checkpointing.
   *                            See the Kinesis Spark Streaming documentation for more
   *                            details on the different types of checkpoints.
   * @param storageLevel Storage level to use for storing the received objects.
   *                     StorageLevel.MEMORY_AND_DISK_2 is recommended.
   * @param awsAccessKeyId  AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
   * @param awsSecretKey  AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
   *
   * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
   * is enabled. Make sure that your checkpoint directory is secure.
   */
  @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
  def createStream(
      jssc: JavaStreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: InitialPositionInStream,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      awsAccessKeyId: String,
      awsSecretKey: String): JavaReceiverInputDStream[Array[Byte]] = {
    createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
      initialPositionInStream, checkpointInterval, storageLevel,
      KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey)
  }

  private def validateRegion(regionName: String): String = {
    Option(RegionUtils.getRegion(regionName)).map { _.getName }.getOrElse {
      throw new IllegalArgumentException(s"Region name '$regionName' is not valid")
    }
  }
}

/**
 * This is a helper class that wraps the methods in KinesisUtils into more Python-friendly class and
 * function so that it can be easily instantiated and called from Python's KinesisUtils.
 */
private class KinesisUtilsPythonHelper {

  def getInitialPositionInStream(initialPositionInStream: Int): InitialPositionInStream = {
    initialPositionInStream match {
      case 0 => InitialPositionInStream.LATEST
      case 1 => InitialPositionInStream.TRIM_HORIZON
      case _ => throw new IllegalArgumentException(
        "Illegal InitialPositionInStream. Please use " +
          "InitialPositionInStream.LATEST or InitialPositionInStream.TRIM_HORIZON")
    }
  }

  // scalastyle:off
  def createStream(
      jssc: JavaStreamingContext,
      kinesisAppName: String,
      streamName: String,
      endpointUrl: String,
      regionName: String,
      initialPositionInStream: Int,
      checkpointInterval: Duration,
      storageLevel: StorageLevel,
      awsAccessKeyId: String,
      awsSecretKey: String,
      stsAssumeRoleArn: String,
      stsSessionName: String,
      stsExternalId: String): JavaReceiverInputDStream[Array[Byte]] = {
    // scalastyle:on
    if (!(stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null)
        && !(stsAssumeRoleArn == null && stsSessionName == null && stsExternalId == null)) {
      throw new IllegalArgumentException("stsAssumeRoleArn, stsSessionName, and stsExtenalId " +
        "must all be defined or all be null")
    }

    if (stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) {
      validateAwsCreds(awsAccessKeyId, awsSecretKey)
      KinesisUtils.createStream(jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
        getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel,
        KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey,
        stsAssumeRoleArn, stsSessionName, stsExternalId)
    } else {
      validateAwsCreds(awsAccessKeyId, awsSecretKey)
      if (awsAccessKeyId == null && awsSecretKey == null) {
        KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName,
          getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel)
      } else {
        KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName,
          getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel,
          awsAccessKeyId, awsSecretKey)
      }
    }
  }

  // Throw IllegalArgumentException unless both values are null or neither are.
  private def validateAwsCreds(awsAccessKeyId: String, awsSecretKey: String) {
    if (awsAccessKeyId == null && awsSecretKey != null) {
      throw new IllegalArgumentException("awsSecretKey is set but awsAccessKeyId is null")
    }
    if (awsAccessKeyId != null && awsSecretKey == null) {
      throw new IllegalArgumentException("awsAccessKeyId is set but awsSecretKey is null")
    }
  }
}