aboutsummaryrefslogtreecommitdiff
path: root/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala
blob: 77553412eda5646851f2ce6d32db4c5adf9d672e (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
/*
 * 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.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
import com.amazonaws.services.kinesis.model.Record

import org.apache.spark.annotation.InterfaceStability
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.{BlockId, StorageLevel}
import org.apache.spark.streaming.{Duration, StreamingContext, Time}
import org.apache.spark.streaming.api.java.JavaStreamingContext
import org.apache.spark.streaming.dstream.ReceiverInputDStream
import org.apache.spark.streaming.receiver.Receiver
import org.apache.spark.streaming.scheduler.ReceivedBlockInfo

private[kinesis] class KinesisInputDStream[T: ClassTag](
    _ssc: StreamingContext,
    val streamName: String,
    val endpointUrl: String,
    val regionName: String,
    val initialPositionInStream: InitialPositionInStream,
    val checkpointAppName: String,
    val checkpointInterval: Duration,
    val _storageLevel: StorageLevel,
    val messageHandler: Record => T,
    val kinesisCreds: SparkAWSCredentials,
    val dynamoDBCreds: Option[SparkAWSCredentials],
    val cloudWatchCreds: Option[SparkAWSCredentials]
  ) extends ReceiverInputDStream[T](_ssc) {

  private[streaming]
  override def createBlockRDD(time: Time, blockInfos: Seq[ReceivedBlockInfo]): RDD[T] = {

    // This returns true even for when blockInfos is empty
    val allBlocksHaveRanges = blockInfos.map { _.metadataOption }.forall(_.nonEmpty)

    if (allBlocksHaveRanges) {
      // Create a KinesisBackedBlockRDD, even when there are no blocks
      val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray
      val seqNumRanges = blockInfos.map {
        _.metadataOption.get.asInstanceOf[SequenceNumberRanges] }.toArray
      val isBlockIdValid = blockInfos.map { _.isBlockIdValid() }.toArray
      logDebug(s"Creating KinesisBackedBlockRDD for $time with ${seqNumRanges.length} " +
          s"seq number ranges: ${seqNumRanges.mkString(", ")} ")
      new KinesisBackedBlockRDD(
        context.sc, regionName, endpointUrl, blockIds, seqNumRanges,
        isBlockIdValid = isBlockIdValid,
        retryTimeoutMs = ssc.graph.batchDuration.milliseconds.toInt,
        messageHandler = messageHandler,
        kinesisCreds = kinesisCreds)
    } else {
      logWarning("Kinesis sequence number information was not present with some block metadata," +
        " it may not be possible to recover from failures")
      super.createBlockRDD(time, blockInfos)
    }
  }

  override def getReceiver(): Receiver[T] = {
    new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
      kinesisCreds, dynamoDBCreds, cloudWatchCreds)
  }
}

@InterfaceStability.Evolving
object KinesisInputDStream {
  /**
   * Builder for [[KinesisInputDStream]] instances.
   *
   * @since 2.2.0
   */
  @InterfaceStability.Evolving
  class Builder {
    // Required params
    private var streamingContext: Option[StreamingContext] = None
    private var streamName: Option[String] = None
    private var checkpointAppName: Option[String] = None

    // Params with defaults
    private var endpointUrl: Option[String] = None
    private var regionName: Option[String] = None
    private var initialPositionInStream: Option[InitialPositionInStream] = None
    private var checkpointInterval: Option[Duration] = None
    private var storageLevel: Option[StorageLevel] = None
    private var kinesisCredsProvider: Option[SparkAWSCredentials] = None
    private var dynamoDBCredsProvider: Option[SparkAWSCredentials] = None
    private var cloudWatchCredsProvider: Option[SparkAWSCredentials] = None

    /**
     * Sets the StreamingContext that will be used to construct the Kinesis DStream. This is a
     * required parameter.
     *
     * @param ssc [[StreamingContext]] used to construct Kinesis DStreams
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def streamingContext(ssc: StreamingContext): Builder = {
      streamingContext = Option(ssc)
      this
    }

    /**
     * Sets the StreamingContext that will be used to construct the Kinesis DStream. This is a
     * required parameter.
     *
     * @param jssc [[JavaStreamingContext]] used to construct Kinesis DStreams
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def streamingContext(jssc: JavaStreamingContext): Builder = {
      streamingContext = Option(jssc.ssc)
      this
    }

    /**
     * Sets the name of the Kinesis stream that the DStream will read from. This is a required
     * parameter.
     *
     * @param streamName Name of Kinesis stream that the DStream will read from
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def streamName(streamName: String): Builder = {
      this.streamName = Option(streamName)
      this
    }

    /**
     * Sets the KCL application name to use when checkpointing state to DynamoDB. This is a
     * required parameter.
     *
     * @param appName Value to use for the KCL app name (used when creating the DynamoDB checkpoint
     *                table and when writing metrics to CloudWatch)
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def checkpointAppName(appName: String): Builder = {
      checkpointAppName = Option(appName)
      this
    }

    /**
     * Sets the AWS Kinesis endpoint URL. Defaults to "https://kinesis.us-east-1.amazonaws.com" if
     * no custom value is specified
     *
     * @param url Kinesis endpoint URL to use
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def endpointUrl(url: String): Builder = {
      endpointUrl = Option(url)
      this
    }

    /**
     * Sets the AWS region to construct clients for. Defaults to "us-east-1" if no custom value
     * is specified.
     *
     * @param regionName Name of AWS region to use (e.g. "us-west-2")
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def regionName(regionName: String): Builder = {
      this.regionName = Option(regionName)
      this
    }

    /**
     * Sets the initial position data is read from in the Kinesis stream. Defaults to
     * [[InitialPositionInStream.LATEST]] if no custom value is specified.
     *
     * @param initialPosition InitialPositionInStream value specifying where Spark Streaming
     *                        will start reading records in the Kinesis stream from
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder = {
      initialPositionInStream = Option(initialPosition)
      this
    }

    /**
     * Sets how often the KCL application state is checkpointed to DynamoDB. Defaults to the Spark
     * Streaming batch interval if no custom value is specified.
     *
     * @param interval [[Duration]] specifying how often the KCL state should be checkpointed to
     *                 DynamoDB.
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def checkpointInterval(interval: Duration): Builder = {
      checkpointInterval = Option(interval)
      this
    }

    /**
     * Sets the storage level of the blocks for the DStream created. Defaults to
     * [[StorageLevel.MEMORY_AND_DISK_2]] if no custom value is specified.
     *
     * @param storageLevel [[StorageLevel]] to use for the DStream data blocks
     * @return Reference to this [[KinesisInputDStream.Builder]]
     */
    def storageLevel(storageLevel: StorageLevel): Builder = {
      this.storageLevel = Option(storageLevel)
      this
    }

    /**
     * Sets the [[SparkAWSCredentials]] to use for authenticating to the AWS Kinesis
     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
     *
     * @param credentials [[SparkAWSCredentials]] to use for Kinesis authentication
     */
    def kinesisCredentials(credentials: SparkAWSCredentials): Builder = {
      kinesisCredsProvider = Option(credentials)
      this
    }

    /**
     * Sets the [[SparkAWSCredentials]] to use for authenticating to the AWS DynamoDB
     * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set.
     *
     * @param credentials [[SparkAWSCredentials]] to use for DynamoDB authentication
     */
    def dynamoDBCredentials(credentials: SparkAWSCredentials): Builder = {
      dynamoDBCredsProvider = Option(credentials)
      this
    }

    /**
     * Sets the [[SparkAWSCredentials]] to use for authenticating to the AWS CloudWatch
     * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set.
     *
     * @param credentials [[SparkAWSCredentials]] to use for CloudWatch authentication
     */
    def cloudWatchCredentials(credentials: SparkAWSCredentials): Builder = {
      cloudWatchCredsProvider = Option(credentials)
      this
    }

    /**
     * Create a new instance of [[KinesisInputDStream]] with configured parameters and the provided
     * message handler.
     *
     * @param handler Function converting [[Record]] instances read by the KCL to DStream type [[T]]
     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
     */
    def buildWithMessageHandler[T: ClassTag](
        handler: Record => T): KinesisInputDStream[T] = {
      val ssc = getRequiredParam(streamingContext, "streamingContext")
      new KinesisInputDStream(
        ssc,
        getRequiredParam(streamName, "streamName"),
        endpointUrl.getOrElse(DEFAULT_KINESIS_ENDPOINT_URL),
        regionName.getOrElse(DEFAULT_KINESIS_REGION_NAME),
        initialPositionInStream.getOrElse(DEFAULT_INITIAL_POSITION_IN_STREAM),
        getRequiredParam(checkpointAppName, "checkpointAppName"),
        checkpointInterval.getOrElse(ssc.graph.batchDuration),
        storageLevel.getOrElse(DEFAULT_STORAGE_LEVEL),
        ssc.sc.clean(handler),
        kinesisCredsProvider.getOrElse(DefaultCredentials),
        dynamoDBCredsProvider,
        cloudWatchCredsProvider)
    }

    /**
     * Create a new instance of [[KinesisInputDStream]] with configured parameters and using the
     * default message handler, which returns [[Array[Byte]]].
     *
     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
     */
    def build(): KinesisInputDStream[Array[Byte]] = buildWithMessageHandler(defaultMessageHandler)

    private def getRequiredParam[T](param: Option[T], paramName: String): T = param.getOrElse {
      throw new IllegalArgumentException(s"No value provided for required parameter $paramName")
    }
  }

  /**
   * Creates a [[KinesisInputDStream.Builder]] for constructing [[KinesisInputDStream]] instances.
   *
   * @since 2.2.0
   *
   * @return [[KinesisInputDStream.Builder]] instance
   */
  def builder: Builder = new Builder

  private[kinesis] def defaultMessageHandler(record: Record): Array[Byte] = {
    if (record == null) return null
    val byteBuffer = record.getData()
    val byteArray = new Array[Byte](byteBuffer.remaining())
    byteBuffer.get(byteArray)
    byteArray
  }

  private[kinesis] val DEFAULT_KINESIS_ENDPOINT_URL: String =
    "https://kinesis.us-east-1.amazonaws.com"
  private[kinesis] val DEFAULT_KINESIS_REGION_NAME: String = "us-east-1"
  private[kinesis] val DEFAULT_INITIAL_POSITION_IN_STREAM: InitialPositionInStream =
    InitialPositionInStream.LATEST
  private[kinesis] val DEFAULT_STORAGE_LEVEL: StorageLevel = StorageLevel.MEMORY_AND_DISK_2
}