You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by budde <gi...@git.apache.org> on 2017/03/10 21:38:02 UTC

[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

GitHub user budde opened a pull request:

    https://github.com/apache/spark/pull/17250

    [SPARK-19911][STREAMING] Add builder interface for Kinesis DStreams

    ## What changes were proposed in this pull request?
    
    - Add new KinesisDStream.scala containing KinesisDStream.Builder class
    - Add KinesisDStreamBuilderSuite test suite
    - Make KinesisInputDStream ctor args package private for testing
    
    ## How was this patch tested?
    
    Added ```KinesisDStreamBuilderSuite``` to verify builder class works as expected

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/budde/spark KinesisStreamBuilder

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/17250.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #17250
    
----
commit 8552caf84dbd5b43a5a8446b10e4ccf9b295f924
Author: Budde <bu...@amazon.com>
Date:   2017-02-23T22:53:50Z

    [SPARK-19911][STREAMING] Add builder interface for Kinesis DStreams
    
    - Add new KinesisDStream.scala containing KinesisDStream.Builder class
    - Add KinesisDStreamBuilderSuite test suite
    - Make KinesisInputDStream ctor args package private for testing

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107761278
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -22,24 +22,28 @@ 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,
    -    streamName: String,
    -    endpointUrl: String,
    -    regionName: String,
    -    initialPositionInStream: InitialPositionInStream,
    -    checkpointAppName: String,
    -    checkpointInterval: Duration,
    -    storageLevel: StorageLevel,
    -    messageHandler: Record => T,
    -    kinesisCredsProvider: SerializableCredentialsProvider
    +    private[kinesis] val streamName: String,
    --- End diff --
    
    If you keep it `val` it should be fine.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105793335
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    +      ssc,
    +      streamName,
    +      endpointUrl,
    +      regionName,
    +      initialPositionInStream,
    +      checkpointAppName,
    +      checkpointInterval,
    +      storageLevel,
    +      messageHandler,
    +      kinesisCredsProvider,
    +      dynamoDBCredsProvider,
    +      cloudWatchCredsProvider)
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] =
    +    new Builder(ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(ssc: StreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(ssc, streamName, KinesisUtils.defaultMessageHandler(_))
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      jssc: JavaStreamingContext,
    --- End diff --
    
    Have the builder take `setStreamingContext`, one which takes `StreamingContext` and the other `JavaStreamingContext`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107748101
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,238 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      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 [[SerialziableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    --- End diff --
    
    ditto


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Found a bit more time to work on this. Changes made:
    
    * Added implementations of ```KinesisDStream.builder()``` that take ```JavaStreamingContext``` instead of ```StreamingContext```
    * Added ```JavaKinesisDStreamBuilderSuite``` to test that the builder interface is accessible in Java
    * Added support for DynamoDB and CloudWatch-specific authentication parameters
    * Made the documentation for ```KinesisDStream.Builder``` more thorough


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75170 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75170/testReport)** for PR 17250 at commit [`5315f1e`](https://github.com/apache/spark/commit/5315f1e9f200be7f97d7838c2d64bc6bfb208ecb).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74351 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74351/testReport)** for PR 17250 at commit [`bcb7667`](https://github.com/apache/spark/commit/bcb76677b9fa58e5b96920d9faaa338183cca26d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107975990
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 com.amazonaws.auth._
    +
    +import org.apache.spark.annotation.InterfaceStability
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * Serializable interface providing a method executors can call to obtain an
    + * AWSCredentialsProvider instance for authenticating to AWS services.
    + */
    +private[kinesis] sealed trait SparkAWSCredentials extends Serializable {
    +  /**
    +   * Return an AWSCredentialProvider instance that can be used by the Kinesis Client
    +   * Library to authenticate to AWS services (Kinesis, CloudWatch and DynamoDB).
    +   */
    +  def provider: AWSCredentialsProvider
    +}
    +
    +/** Returns DefaultAWSCredentialsProviderChain for authentication. */
    +private[kinesis] final case object DefaultCredentials extends SparkAWSCredentials {
    +
    +  def provider: AWSCredentialsProvider = new DefaultAWSCredentialsProviderChain
    +}
    +
    +/**
    + * Returns AWSStaticCredentialsProvider constructed using basic AWS keypair. Falls back to using
    + * DefaultCredentialsProviderChain if unable to construct a AWSCredentialsProviderChain
    + * instance with the provided arguments (e.g. if they are null).
    + */
    +private[kinesis] final case class BasicCredentials(
    +    awsAccessKeyId: String,
    +    awsSecretKey: String) extends SparkAWSCredentials with Logging {
    +
    +  def provider: AWSCredentialsProvider = try {
    +    new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKeyId, awsSecretKey))
    +  } catch {
    +    case e: IllegalArgumentException =>
    +      logWarning("Unable to construct AWSStaticCredentialsProvider with provided keypair; " +
    +        "falling back to DefaultCredentialsProviderChain.", e)
    +      new DefaultAWSCredentialsProviderChain
    +  }
    +}
    +
    +/**
    + * Returns an STSAssumeRoleSessionCredentialsProvider instance which assumes an IAM
    + * role in order to authenticate against resources in an external account.
    + */
    +private[kinesis] final case class STSCredentials(
    +    stsRoleArn: String,
    +    stsSessionName: String,
    +    stsExternalId: Option[String] = None,
    +    longLivedCreds: SparkAWSCredentials = DefaultCredentials)
    +  extends SparkAWSCredentials  {
    +
    +  def provider: AWSCredentialsProvider = {
    +    val builder = new STSAssumeRoleSessionCredentialsProvider.Builder(stsRoleArn, stsSessionName)
    +      .withLongLivedCredentialsProvider(longLivedCreds.provider)
    +    stsExternalId match {
    +      case Some(stsExternalId) =>
    +        builder.withExternalId(stsExternalId)
    +          .build()
    +      case None =>
    +        builder.build()
    +    }
    +  }
    +}
    +
    +@InterfaceStability.Evolving
    +object SparkAWSCredentials {
    +  /**
    +   * Builder for [[SparkAWSCredentials]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Evolving
    +  class Builder {
    +    private var basicCreds: Option[BasicCredentials] = None
    +    private var stsCreds: Option[STSCredentials] = None
    +
    +    /**
    +     * Use a basic AWS keypair for long-lived authorization.
    +     *
    +     * @note The given AWS keypair will be saved in DStream checkpoints if checkpointing is
    +     * enabled. Make sure that your checkpoint directory is secure if using the default
    --- End diff --
    
    The link in this case will be quite long-- URL just by itself pushes it over the 100 char limit:
    
    ```[[http://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default default credential provider chain]]```
    
    Do you know if there's a way to safely split this into multiple lines? Should I just turn style checks off for this comment?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107747493
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -22,24 +22,28 @@ 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,
    -    streamName: String,
    -    endpointUrl: String,
    -    regionName: String,
    -    initialPositionInStream: InitialPositionInStream,
    -    checkpointAppName: String,
    -    checkpointInterval: Duration,
    -    storageLevel: StorageLevel,
    -    messageHandler: Record => T,
    -    kinesisCredsProvider: SerializableCredentialsProvider
    +    private[kinesis] val streamName: String,
    --- End diff --
    
    The `private`s are unnecessary since the class itself is private. You can either:
     1) remove `private[kinesis]`
     2) make it `private val` and then use `PrivateMethodTester` in the tests.
    
    I'm fine either way.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz Updated the PR to remove ```defaultMessageHandler()``` from ```KinesisUtils``` in favor of keeping this method in ```KinesisInputDStream```. My thought here was that this would be a better place for it since we've put ```KinesisUtils``` on the deprecation path.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107318150
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    +object KinesisInputDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  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[SerializableCredentialsProvider] = None
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = 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 [[SerializableCredentialsProvider]] to use for authenticating to the AWS Kinesis
    +     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
    +     *
    +     * @param credentials [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    +     */
    +    def kinesisCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      kinesisCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    +     */
    +    def dynamoDBCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      dynamoDBCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    +     */
    +    def cloudWatchCredentials(credentials: SerializableCredentialsProvider): 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),
    +        handler,
    +        kinesisCredsProvider.getOrElse(DefaultCredentialsProvider),
    +        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")
    +    }
    +
    +    private def buildSTSCredentials(
    --- End diff --
    
    You're right, thanks for catching it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107224819
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    +object KinesisInputDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  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[SerializableCredentialsProvider] = None
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = 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 [[SerializableCredentialsProvider]] to use for authenticating to the AWS Kinesis
    +     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
    +     *
    +     * @param credentials [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    +     */
    +    def kinesisCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      kinesisCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    +     */
    +    def dynamoDBCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      dynamoDBCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    +     */
    +    def cloudWatchCredentials(credentials: SerializableCredentialsProvider): 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),
    +        handler,
    +        kinesisCredsProvider.getOrElse(DefaultCredentialsProvider),
    +        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")
    +    }
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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
    --- End diff --
    
    nit: move `=` to the line above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107748068
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,238 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      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 [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    --- End diff --
    
    `SparkAWSCredentials`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75111/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107294998
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    +  /**
    +   * Builder for [[SerializableCredentialsProvider]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder {
    +    private var awsAccessKeyId: Option[String] = None
    +    private var awsSecretKey: Option[String] = None
    +    private var stsRoleArn: Option[String] = None
    +    private var stsSessionName: Option[String] = None
    +    private var stsExternalId: Option[String] = None
    +
    +
    +    /**
    +     * Sets the AWS access key ID when using a basic AWS keypair for long-lived authorization
    +     * credentials. A value must also be provided for the AWS secret key.
    +     *
    +     * @param accessKeyId AWS access key ID
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def awsAccessKeyId(accessKeyId: String): Builder = {
    --- End diff --
    
    anyone who provides an `accessKeyId` should also provide a `secretKey` therefore I would take both together.
    
    `.withKeys(awsAccessKey: String, awsSecretKey: String)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74342 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74342/testReport)** for PR 17250 at commit [`8552caf`](https://github.com/apache/spark/commit/8552caf84dbd5b43a5a8446b10e4ccf9b295f924).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class Builder[T: ClassTag](`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107749751
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 java.lang.IllegalArgumentException
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Seconds, StreamingContext, TestSuiteBase}
    +
    +class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterEach
    +   with MockitoSugar {
    +  import KinesisInputDStream._
    +
    +  private val ssc = new StreamingContext(conf, batchDuration)
    --- End diff --
    
    since we're not really starting a stream, and testing API's, we should just mock it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74342/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74342 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74342/testReport)** for PR 17250 at commit [`8552caf`](https://github.com/apache/spark/commit/8552caf84dbd5b43a5a8446b10e4ccf9b295f924).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz I think if we're eliminating the constructor arguments then the second approach you've proposed might make more sense. I can't think of anything cleaner.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107296247
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    --- End diff --
    
    I wouldn't call these `Stable` just yet :) Let's call it evolving for one release cycle


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107296962
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    +object KinesisInputDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  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[SerializableCredentialsProvider] = None
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = 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 [[SerializableCredentialsProvider]] to use for authenticating to the AWS Kinesis
    +     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
    +     *
    +     * @param credentials [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    +     */
    +    def kinesisCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      kinesisCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    +     */
    +    def dynamoDBCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      dynamoDBCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    +     */
    +    def cloudWatchCredentials(credentials: SerializableCredentialsProvider): 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),
    +        handler,
    +        kinesisCredsProvider.getOrElse(DefaultCredentialsProvider),
    +        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")
    +    }
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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] = {
    --- End diff --
    
    sounds good


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107760106
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -22,24 +22,28 @@ 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,
    -    streamName: String,
    -    endpointUrl: String,
    -    regionName: String,
    -    initialPositionInStream: InitialPositionInStream,
    -    checkpointAppName: String,
    -    checkpointInterval: Duration,
    -    storageLevel: StorageLevel,
    -    messageHandler: Record => T,
    -    kinesisCredsProvider: SerializableCredentialsProvider
    +    private[kinesis] val streamName: String,
    --- End diff --
    
    Yeah, it's really just there so that I could access the values directly from the test. I'll look into using ```PrivateMethodTester```, thanks for the suggestion


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74351 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74351/testReport)** for PR 17250 at commit [`bcb7667`](https://github.com/apache/spark/commit/bcb76677b9fa58e5b96920d9faaa338183cca26d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class Builder[T: ClassTag](`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107750493
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 org.apache.spark.streaming.TestSuiteBase
    +import org.apache.spark.util.Utils
    +
    +class SparkAWSCredentialsBuilderSuite extends TestSuiteBase {
    +  private def builder = SparkAWSCredentials.builder
    +
    +  private val basicCreds = BasicCredentials(
    +    awsAccessKeyId = "a-very-nice-access-key",
    +    awsSecretKey = "a-very-nice-secret-key")
    +
    +  private val stsCreds = STSCredentials(
    +    stsRoleArn = "a-very-nice-role-arn",
    +    stsSessionName = "a-very-nice-secret-key",
    +    stsExternalId = Option("a-very-nice-external-id"),
    +    longLivedCredsProvider = basicCreds)
    +
    +  test("should build DefaultCredentials when given no params") {
    +    assert(builder.build() == DefaultCredentials)
    +  }
    +
    +  test("should build BasicCredentials") {
    +    assertResult(basicCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +  }
    +
    +  test("should build STSCredentials") {
    +    // No external ID, default long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None,
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .build()
    +    }
    +    // Default long-lived creds
    +    assertResult(stsCreds.copy(
    +      longLivedCredsProvider = DefaultCredentials)) {
    --- End diff --
    
    nit: single line please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @budde Do you think you can update this PR? The 2.2 branch will be cut on Monday (2017-03-18).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105792446
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    --- End diff --
    
    We shouldn't default to the Spark app name


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74901/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105796830
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    --- End diff --
    
    With this many methods for the credential provider, I feel we need a credential provider builder. I wouldn't want to re-enter everything again between `dynamoDb` and `cloudWatch`. If I want to keep them separate from the `kinesis` credentials


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74418 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74418/testReport)** for PR 17250 at commit [`8aeef08`](https://github.com/apache/spark/commit/8aeef08410999910e864d1de07f6eae21fd5409a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75065 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75065/testReport)** for PR 17250 at commit [`337b6ba`](https://github.com/apache/spark/commit/337b6ba575c969417566ee65664c93916e6923d4).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105792656
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    --- End diff --
    
    Kinda feels unnecessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107764949
  
    --- Diff: external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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 org.junit.Test;
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
    +
    +import org.apache.spark.storage.StorageLevel;
    +import org.apache.spark.streaming.Duration;
    +import org.apache.spark.streaming.Seconds;
    +import org.apache.spark.streaming.LocalJavaStreamingContext;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +
    +public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext {
    +  /**
    +   * Basic test to ensure that the KinesisDStream.Builder interface is accessible from Java.
    +   */
    +  @Test
    +  public void testJavaKinesisDStreamBuilder() {
    +    String streamName = "a-very-nice-stream-name";
    +    String endpointUrl = "https://kinesis.us-west-2.amazonaws.com";
    +    String region = "us-west-2";
    +    InitialPositionInStream initialPosition = InitialPositionInStream.TRIM_HORIZON;
    +    String appName = "a-very-nice-kinesis-app";
    +    Duration checkpointInterval = Seconds.apply(30);
    +    StorageLevel storageLevel = StorageLevel.MEMORY_ONLY();
    +
    +    KinesisInputDStream<byte[]> kinesisDStream = KinesisInputDStream.builder()
    +      .streamingContext(ssc)
    +      .streamName(streamName)
    +      .endpointUrl(endpointUrl)
    +      .regionName(region)
    +      .initialPositionInStream(initialPosition)
    +      .checkpointAppName(appName)
    +      .checkpointInterval(checkpointInterval)
    +      .storageLevel(storageLevel)
    +      .build();
    +    assert(kinesisDStream.streamName() == streamName);
    +    assert(kinesisDStream.endpointUrl() == endpointUrl);
    +    assert(kinesisDStream.regionName() == region);
    +    assert(kinesisDStream.initialPositionInStream() == initialPosition);
    +    assert(kinesisDStream.checkpointAppName() == appName);
    +    assert(kinesisDStream.checkpointInterval() == checkpointInterval);
    +    assert(kinesisDStream._storageLevel() == storageLevel);
    +    ssc.stop();
    --- End diff --
    
    Using a mock here and in the other test might not be very practical after all-- looks like the ```DStream``` constructor hooks into ```StreamingContext```. We would at least need to mock it's [```getState()``` method](https://github.com/apache/spark/blob/d5b1d5fc80153571c308130833d0c0774de62c92/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala#L220) as well as [mocking a ```SparkContext``` along with its local properties](https://github.com/apache/spark/blob/d5b1d5fc80153571c308130833d0c0774de62c92/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala#L220).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107759722
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 org.apache.spark.streaming.TestSuiteBase
    +import org.apache.spark.util.Utils
    +
    +class SparkAWSCredentialsBuilderSuite extends TestSuiteBase {
    +  private def builder = SparkAWSCredentials.builder
    +
    +  private val basicCreds = BasicCredentials(
    +    awsAccessKeyId = "a-very-nice-access-key",
    +    awsSecretKey = "a-very-nice-secret-key")
    +
    +  private val stsCreds = STSCredentials(
    +    stsRoleArn = "a-very-nice-role-arn",
    +    stsSessionName = "a-very-nice-secret-key",
    +    stsExternalId = Option("a-very-nice-external-id"),
    +    longLivedCredsProvider = basicCreds)
    +
    +  test("should build DefaultCredentials when given no params") {
    +    assert(builder.build() == DefaultCredentials)
    +  }
    +
    +  test("should build BasicCredentials") {
    +    assertResult(basicCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +  }
    +
    +  test("should build STSCredentials") {
    +    // No external ID, default long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None,
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .build()
    +    }
    +    // Default long-lived creds
    +    assertResult(stsCreds.copy(
    +      longLivedCredsProvider = DefaultCredentials)) {
    --- End diff --
    
    I'll fix it. This happened since ```DefaultCredentialsProvider``` was shortened to ```DefaultCredentials``` so I'll try to check for other places where a multiline statement can be rolled up into a single line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75172/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75111 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75111/testReport)** for PR 17250 at commit [`6f11978`](https://github.com/apache/spark/commit/6f11978326e59e4d383d7a64eef981b917364af6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    LGTM pending tests. Thanks a lot for this PR @budde
    
    PS  It's okay to make new commits, you don't have to squash commit every time :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105792395
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = regionName
    +      this
    +    }
    +
    +    /**
    +     * Sets the initial position data is read from in the Kinesis stream. Defaults to
    +     * InitialPositionInStream.LATEST if no custom value is specified.
    --- End diff --
    
    could you link `InitialPositionInStream` for simplicity


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105795516
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -31,15 +31,17 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo
     
     private[kinesis] class KinesisInputDStream[T: ClassTag](
         _ssc: StreamingContext,
    -    streamName: String,
    -    endpointUrl: String,
    -    regionName: String,
    -    initialPositionInStream: InitialPositionInStream,
    -    checkpointAppName: String,
    -    checkpointInterval: Duration,
    -    storageLevel: StorageLevel,
    -    messageHandler: Record => T,
    -    kinesisCredsProvider: SerializableCredentialsProvider
    +    private[kinesis] val streamName: String,
    +    private[kinesis] val endpointUrl: String,
    +    private[kinesis] val regionName: String,
    +    private[kinesis] val initialPositionInStream: InitialPositionInStream,
    +    private[kinesis] val checkpointAppName: String,
    +    private[kinesis] val checkpointInterval: Duration,
    +    private[kinesis] val _storageLevel: StorageLevel,
    --- End diff --
    
    why the underscore?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806322
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    +      ssc,
    +      streamName,
    +      endpointUrl,
    +      regionName,
    +      initialPositionInStream,
    +      checkpointAppName,
    +      checkpointInterval,
    +      storageLevel,
    +      messageHandler,
    +      kinesisCredsProvider,
    +      dynamoDBCredsProvider,
    +      cloudWatchCredsProvider)
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] =
    +    new Builder(ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(ssc: StreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(ssc, streamName, KinesisUtils.defaultMessageHandler(_))
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      jssc: JavaStreamingContext,
    --- End diff --
    
    Will be made required builder arguments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107759778
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 org.apache.spark.streaming.TestSuiteBase
    +import org.apache.spark.util.Utils
    +
    +class SparkAWSCredentialsBuilderSuite extends TestSuiteBase {
    +  private def builder = SparkAWSCredentials.builder
    +
    +  private val basicCreds = BasicCredentials(
    +    awsAccessKeyId = "a-very-nice-access-key",
    +    awsSecretKey = "a-very-nice-secret-key")
    +
    +  private val stsCreds = STSCredentials(
    +    stsRoleArn = "a-very-nice-role-arn",
    +    stsSessionName = "a-very-nice-secret-key",
    +    stsExternalId = Option("a-very-nice-external-id"),
    +    longLivedCredsProvider = basicCreds)
    +
    +  test("should build DefaultCredentials when given no params") {
    +    assert(builder.build() == DefaultCredentials)
    +  }
    +
    +  test("should build BasicCredentials") {
    +    assertResult(basicCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +  }
    +
    +  test("should build STSCredentials") {
    +    // No external ID, default long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None,
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .build()
    +    }
    +    // Default long-lived creds
    +    assertResult(stsCreds.copy(
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .build()
    +    }
    +    // No external ID, basic keypair for long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +    // Basic keypair for long-lived creds
    +    assertResult(stsCreds) {
    +      builder.stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +    // Order shouldn't matter
    +    assertResult(stsCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .build()
    +    }
    +  }
    +
    +  test("SparkAWSCredentials classes should be serializable") {
    +    Utils.deserialize[BasicCredentials](
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz A conference took up a lot of my time last week but I should have it updated later today


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107318178
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74418 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74418/testReport)** for PR 17250 at commit [`8aeef08`](https://github.com/apache/spark/commit/8aeef08410999910e864d1de07f6eae21fd5409a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class Builder[T: ClassTag](`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz Awesome, thanks for reviewing this!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107759339
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 java.lang.IllegalArgumentException
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Seconds, StreamingContext, TestSuiteBase}
    +
    +class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterEach
    +   with MockitoSugar {
    +  import KinesisInputDStream._
    +
    +  private val ssc = new StreamingContext(conf, batchDuration)
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107976590
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 com.amazonaws.auth._
    +
    +import org.apache.spark.annotation.InterfaceStability
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * Serializable interface providing a method executors can call to obtain an
    + * AWSCredentialsProvider instance for authenticating to AWS services.
    + */
    +private[kinesis] sealed trait SparkAWSCredentials extends Serializable {
    +  /**
    +   * Return an AWSCredentialProvider instance that can be used by the Kinesis Client
    +   * Library to authenticate to AWS services (Kinesis, CloudWatch and DynamoDB).
    +   */
    +  def provider: AWSCredentialsProvider
    +}
    +
    +/** Returns DefaultAWSCredentialsProviderChain for authentication. */
    +private[kinesis] final case object DefaultCredentials extends SparkAWSCredentials {
    +
    +  def provider: AWSCredentialsProvider = new DefaultAWSCredentialsProviderChain
    +}
    +
    +/**
    + * Returns AWSStaticCredentialsProvider constructed using basic AWS keypair. Falls back to using
    + * DefaultCredentialsProviderChain if unable to construct a AWSCredentialsProviderChain
    + * instance with the provided arguments (e.g. if they are null).
    + */
    +private[kinesis] final case class BasicCredentials(
    +    awsAccessKeyId: String,
    +    awsSecretKey: String) extends SparkAWSCredentials with Logging {
    +
    +  def provider: AWSCredentialsProvider = try {
    +    new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKeyId, awsSecretKey))
    +  } catch {
    +    case e: IllegalArgumentException =>
    +      logWarning("Unable to construct AWSStaticCredentialsProvider with provided keypair; " +
    +        "falling back to DefaultCredentialsProviderChain.", e)
    +      new DefaultAWSCredentialsProviderChain
    +  }
    +}
    +
    +/**
    + * Returns an STSAssumeRoleSessionCredentialsProvider instance which assumes an IAM
    + * role in order to authenticate against resources in an external account.
    + */
    +private[kinesis] final case class STSCredentials(
    +    stsRoleArn: String,
    +    stsSessionName: String,
    +    stsExternalId: Option[String] = None,
    +    longLivedCreds: SparkAWSCredentials = DefaultCredentials)
    +  extends SparkAWSCredentials  {
    +
    +  def provider: AWSCredentialsProvider = {
    +    val builder = new STSAssumeRoleSessionCredentialsProvider.Builder(stsRoleArn, stsSessionName)
    +      .withLongLivedCredentialsProvider(longLivedCreds.provider)
    +    stsExternalId match {
    +      case Some(stsExternalId) =>
    +        builder.withExternalId(stsExternalId)
    +          .build()
    +      case None =>
    +        builder.build()
    +    }
    +  }
    +}
    +
    +@InterfaceStability.Evolving
    +object SparkAWSCredentials {
    +  /**
    +   * Builder for [[SparkAWSCredentials]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Evolving
    +  class Builder {
    +    private var basicCreds: Option[BasicCredentials] = None
    +    private var stsCreds: Option[STSCredentials] = None
    +
    +    /**
    +     * Use a basic AWS keypair for long-lived authorization.
    +     *
    +     * @note The given AWS keypair will be saved in DStream checkpoints if checkpointing is
    +     * enabled. Make sure that your checkpoint directory is secure if using the default
    --- End diff --
    
    Feel free to add
    ```
    // scalastyle: off
    
    // scalastyle: on
    ```
    around the doc


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75172 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75172/testReport)** for PR 17250 at commit [`03f91da`](https://github.com/apache/spark/commit/03f91dadb3878aa47f2a134e3e1b8d46aadd3b47).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75111 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75111/testReport)** for PR 17250 at commit [`6f11978`](https://github.com/apache/spark/commit/6f11978326e59e4d383d7a64eef981b917364af6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74909/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/17250


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105793124
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    --- End diff --
    
    I would get rid of the constructor params, and check for required values here, i.e.
     1. `ssc`
     2. `streamName`
     3. `appName`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75065/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806364
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * Sets the AWS Kinesis endpoint URL. Defaults to "https://kinesis.us-east-1.amazonaws.com" if
    +     * no custom value is specified
    --- End diff --
    
    I'll take a look and see. To be honest supplying both of these has always felt pretty redundant to me. The AWS SDK has changed a bit in how it handles endpoints and regions as well, so it may also be worth revisiting how ```KinesisReceiver``` uses these params.
    
    Long term, it may also be nice to allow for different endpoints to be specified for Kinesis, DynamoDB and CloudWatch (I think the KCL should support this...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107318164
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    --- End diff --
    
    For sure. Honestly I just cribbed these annotations from ```SparkSession.Builder``` so I appreciate you letting me know what the proper convention is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806316
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    +      ssc,
    +      streamName,
    +      endpointUrl,
    +      regionName,
    +      initialPositionInStream,
    +      checkpointAppName,
    +      checkpointInterval,
    +      storageLevel,
    +      messageHandler,
    +      kinesisCredsProvider,
    +      dynamoDBCredsProvider,
    +      cloudWatchCredsProvider)
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] =
    +    new Builder(ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(ssc: StreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(ssc, streamName, KinesisUtils.defaultMessageHandler(_))
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      jssc: JavaStreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] = builder(jssc.ssc, streamName, messageHandler)
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806333
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    --- End diff --
    
    I'll look at introducing a credential builder


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105795423
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    +      ssc,
    +      streamName,
    +      endpointUrl,
    +      regionName,
    +      initialPositionInStream,
    +      checkpointAppName,
    +      checkpointInterval,
    +      storageLevel,
    +      messageHandler,
    +      kinesisCredsProvider,
    +      dynamoDBCredsProvider,
    +      cloudWatchCredsProvider)
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] =
    +    new Builder(ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(ssc: StreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(ssc, streamName, KinesisUtils.defaultMessageHandler(_))
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      jssc: JavaStreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] = builder(jssc.ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(jssc: JavaStreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(jssc.ssc, streamName)
    +
    +  private[kinesis] val DEFAULT_KINESIS_ENDPOINT_URL: String
    +    = "https://kinesis.us-east-1.amazonaws.com"
    --- End diff --
    
    nit: move `=` to line above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107971313
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * 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 com.amazonaws.auth._
    +
    +import org.apache.spark.annotation.InterfaceStability
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * Serializable interface providing a method executors can call to obtain an
    + * AWSCredentialsProvider instance for authenticating to AWS services.
    + */
    +private[kinesis] sealed trait SparkAWSCredentials extends Serializable {
    +  /**
    +   * Return an AWSCredentialProvider instance that can be used by the Kinesis Client
    +   * Library to authenticate to AWS services (Kinesis, CloudWatch and DynamoDB).
    +   */
    +  def provider: AWSCredentialsProvider
    +}
    +
    +/** Returns DefaultAWSCredentialsProviderChain for authentication. */
    +private[kinesis] final case object DefaultCredentials extends SparkAWSCredentials {
    +
    +  def provider: AWSCredentialsProvider = new DefaultAWSCredentialsProviderChain
    +}
    +
    +/**
    + * Returns AWSStaticCredentialsProvider constructed using basic AWS keypair. Falls back to using
    + * DefaultCredentialsProviderChain if unable to construct a AWSCredentialsProviderChain
    + * instance with the provided arguments (e.g. if they are null).
    + */
    +private[kinesis] final case class BasicCredentials(
    +    awsAccessKeyId: String,
    +    awsSecretKey: String) extends SparkAWSCredentials with Logging {
    +
    +  def provider: AWSCredentialsProvider = try {
    +    new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKeyId, awsSecretKey))
    +  } catch {
    +    case e: IllegalArgumentException =>
    +      logWarning("Unable to construct AWSStaticCredentialsProvider with provided keypair; " +
    +        "falling back to DefaultCredentialsProviderChain.", e)
    +      new DefaultAWSCredentialsProviderChain
    +  }
    +}
    +
    +/**
    + * Returns an STSAssumeRoleSessionCredentialsProvider instance which assumes an IAM
    + * role in order to authenticate against resources in an external account.
    + */
    +private[kinesis] final case class STSCredentials(
    +    stsRoleArn: String,
    +    stsSessionName: String,
    +    stsExternalId: Option[String] = None,
    +    longLivedCreds: SparkAWSCredentials = DefaultCredentials)
    +  extends SparkAWSCredentials  {
    +
    +  def provider: AWSCredentialsProvider = {
    +    val builder = new STSAssumeRoleSessionCredentialsProvider.Builder(stsRoleArn, stsSessionName)
    +      .withLongLivedCredentialsProvider(longLivedCreds.provider)
    +    stsExternalId match {
    +      case Some(stsExternalId) =>
    +        builder.withExternalId(stsExternalId)
    +          .build()
    +      case None =>
    +        builder.build()
    +    }
    +  }
    +}
    +
    +@InterfaceStability.Evolving
    +object SparkAWSCredentials {
    +  /**
    +   * Builder for [[SparkAWSCredentials]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Evolving
    +  class Builder {
    +    private var basicCreds: Option[BasicCredentials] = None
    +    private var stsCreds: Option[STSCredentials] = None
    +
    +    /**
    +     * Use a basic AWS keypair for long-lived authorization.
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107318157
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    +object KinesisInputDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  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[SerializableCredentialsProvider] = None
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = 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 [[SerializableCredentialsProvider]] to use for authenticating to the AWS Kinesis
    +     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
    +     *
    +     * @param credentials [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    +     */
    +    def kinesisCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      kinesisCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    +     */
    +    def dynamoDBCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      dynamoDBCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    +     */
    +    def cloudWatchCredentials(credentials: SerializableCredentialsProvider): 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),
    +        handler,
    +        kinesisCredsProvider.getOrElse(DefaultCredentialsProvider),
    +        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")
    +    }
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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
    --- End diff --
    
    Will fix. Sorry I keep doing this :-/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107294828
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    --- End diff --
    
    hmm. This isn't a great name for a user facing API, the user shouldn't have to care about if the provider is serializable or not, that's an implementation detail.
    
    I understand your concerns with the `AWSCredentials` name collisions. However, I think it's the best name there is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806353
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = regionName
    +      this
    +    }
    +
    +    /**
    +     * Sets the initial position data is read from in the Kinesis stream. Defaults to
    +     * InitialPositionInStream.LATEST if no custom value is specified.
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105793427
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    +      ssc,
    +      streamName,
    +      endpointUrl,
    +      regionName,
    +      initialPositionInStream,
    +      checkpointAppName,
    +      checkpointInterval,
    +      storageLevel,
    +      messageHandler,
    +      kinesisCredsProvider,
    +      dynamoDBCredsProvider,
    +      cloudWatchCredsProvider)
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] =
    +    new Builder(ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(ssc: StreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(ssc, streamName, KinesisUtils.defaultMessageHandler(_))
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      jssc: JavaStreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] = builder(jssc.ssc, streamName, messageHandler)
    --- End diff --
    
    let's also have a `setMessageHandler` function as well


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107758905
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,238 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      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 [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    --- End diff --
    
    My bad, thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz Updated per your feedback. Most significant change is renaming ```SerializableCredentialsProvider``` to ```SparkAWSCredentials``` (as well as renaming its subclasses) and refactoring its builder as you've suggested.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107759287
  
    --- Diff: external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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 org.junit.Test;
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
    +
    +import org.apache.spark.storage.StorageLevel;
    +import org.apache.spark.streaming.Duration;
    +import org.apache.spark.streaming.Seconds;
    +import org.apache.spark.streaming.LocalJavaStreamingContext;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +
    +public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext {
    +  /**
    +   * Basic test to ensure that the KinesisDStream.Builder interface is accessible from Java.
    +   */
    +  @Test
    +  public void testJavaKinesisDStreamBuilder() {
    +    String streamName = "a-very-nice-stream-name";
    +    String endpointUrl = "https://kinesis.us-west-2.amazonaws.com";
    +    String region = "us-west-2";
    +    InitialPositionInStream initialPosition = InitialPositionInStream.TRIM_HORIZON;
    +    String appName = "a-very-nice-kinesis-app";
    +    Duration checkpointInterval = Seconds.apply(30);
    +    StorageLevel storageLevel = StorageLevel.MEMORY_ONLY();
    +
    +    KinesisInputDStream<byte[]> kinesisDStream = KinesisInputDStream.builder()
    +      .streamingContext(ssc)
    +      .streamName(streamName)
    +      .endpointUrl(endpointUrl)
    +      .regionName(region)
    +      .initialPositionInStream(initialPosition)
    +      .checkpointAppName(appName)
    +      .checkpointInterval(checkpointInterval)
    +      .storageLevel(storageLevel)
    +      .build();
    +    assert(kinesisDStream.streamName() == streamName);
    +    assert(kinesisDStream.endpointUrl() == endpointUrl);
    +    assert(kinesisDStream.regionName() == region);
    +    assert(kinesisDStream.initialPositionInStream() == initialPosition);
    +    assert(kinesisDStream.checkpointAppName() == appName);
    +    assert(kinesisDStream.checkpointInterval() == checkpointInterval);
    +    assert(kinesisDStream._storageLevel() == storageLevel);
    +    ssc.stop();
    --- End diff --
    
    I'll probably just go the mock route then and ignore the context all together. I was seeing a bunch of "Spark context is already running" error messages when I tried to run all of the streaming tests before I added this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806378
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    --- End diff --
    
    This is probably the first builder class I've implemented so I'll defer to your judgment here :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806308
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -31,15 +31,17 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo
     
     private[kinesis] class KinesisInputDStream[T: ClassTag](
         _ssc: StreamingContext,
    -    streamName: String,
    -    endpointUrl: String,
    -    regionName: String,
    -    initialPositionInStream: InitialPositionInStream,
    -    checkpointAppName: String,
    -    checkpointInterval: Duration,
    -    storageLevel: StorageLevel,
    -    messageHandler: Record => T,
    -    kinesisCredsProvider: SerializableCredentialsProvider
    +    private[kinesis] val streamName: String,
    +    private[kinesis] val endpointUrl: String,
    +    private[kinesis] val regionName: String,
    +    private[kinesis] val initialPositionInStream: InitialPositionInStream,
    +    private[kinesis] val checkpointAppName: String,
    +    private[kinesis] val checkpointInterval: Duration,
    +    private[kinesis] val _storageLevel: StorageLevel,
    --- End diff --
    
    Declaring ```val storageLevel``` collides with [```DStream.storageLevel```](https://github.com/apache/spark/blob/d5b1d5fc80153571c308130833d0c0774de62c92/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala#L96)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74379 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74379/testReport)** for PR 17250 at commit [`a604dc5`](https://github.com/apache/spark/commit/a604dc5952a1c9939d433371abc969670a8ff6ab).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74379 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74379/testReport)** for PR 17250 at commit [`a604dc5`](https://github.com/apache/spark/commit/a604dc5952a1c9939d433371abc969670a8ff6ab).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class Builder[T: ClassTag](`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107977628
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 com.amazonaws.auth._
    +
    +import org.apache.spark.annotation.InterfaceStability
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * Serializable interface providing a method executors can call to obtain an
    + * AWSCredentialsProvider instance for authenticating to AWS services.
    + */
    +private[kinesis] sealed trait SparkAWSCredentials extends Serializable {
    +  /**
    +   * Return an AWSCredentialProvider instance that can be used by the Kinesis Client
    +   * Library to authenticate to AWS services (Kinesis, CloudWatch and DynamoDB).
    +   */
    +  def provider: AWSCredentialsProvider
    +}
    +
    +/** Returns DefaultAWSCredentialsProviderChain for authentication. */
    +private[kinesis] final case object DefaultCredentials extends SparkAWSCredentials {
    +
    +  def provider: AWSCredentialsProvider = new DefaultAWSCredentialsProviderChain
    +}
    +
    +/**
    + * Returns AWSStaticCredentialsProvider constructed using basic AWS keypair. Falls back to using
    + * DefaultCredentialsProviderChain if unable to construct a AWSCredentialsProviderChain
    + * instance with the provided arguments (e.g. if they are null).
    + */
    +private[kinesis] final case class BasicCredentials(
    +    awsAccessKeyId: String,
    +    awsSecretKey: String) extends SparkAWSCredentials with Logging {
    +
    +  def provider: AWSCredentialsProvider = try {
    +    new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKeyId, awsSecretKey))
    +  } catch {
    +    case e: IllegalArgumentException =>
    +      logWarning("Unable to construct AWSStaticCredentialsProvider with provided keypair; " +
    +        "falling back to DefaultCredentialsProviderChain.", e)
    +      new DefaultAWSCredentialsProviderChain
    +  }
    +}
    +
    +/**
    + * Returns an STSAssumeRoleSessionCredentialsProvider instance which assumes an IAM
    + * role in order to authenticate against resources in an external account.
    + */
    +private[kinesis] final case class STSCredentials(
    +    stsRoleArn: String,
    +    stsSessionName: String,
    +    stsExternalId: Option[String] = None,
    +    longLivedCreds: SparkAWSCredentials = DefaultCredentials)
    +  extends SparkAWSCredentials  {
    +
    +  def provider: AWSCredentialsProvider = {
    +    val builder = new STSAssumeRoleSessionCredentialsProvider.Builder(stsRoleArn, stsSessionName)
    +      .withLongLivedCredentialsProvider(longLivedCreds.provider)
    +    stsExternalId match {
    +      case Some(stsExternalId) =>
    +        builder.withExternalId(stsExternalId)
    +          .build()
    +      case None =>
    +        builder.build()
    +    }
    +  }
    +}
    +
    +@InterfaceStability.Evolving
    +object SparkAWSCredentials {
    +  /**
    +   * Builder for [[SparkAWSCredentials]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Evolving
    +  class Builder {
    +    private var basicCreds: Option[BasicCredentials] = None
    +    private var stsCreds: Option[STSCredentials] = None
    +
    +    /**
    +     * Use a basic AWS keypair for long-lived authorization.
    +     *
    +     * @note The given AWS keypair will be saved in DStream checkpoints if checkpointing is
    +     * enabled. Make sure that your checkpoint directory is secure if using the default
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107768752
  
    --- Diff: external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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 org.junit.Test;
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
    +
    +import org.apache.spark.storage.StorageLevel;
    +import org.apache.spark.streaming.Duration;
    +import org.apache.spark.streaming.Seconds;
    +import org.apache.spark.streaming.LocalJavaStreamingContext;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +
    +public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext {
    +  /**
    +   * Basic test to ensure that the KinesisDStream.Builder interface is accessible from Java.
    +   */
    +  @Test
    +  public void testJavaKinesisDStreamBuilder() {
    +    String streamName = "a-very-nice-stream-name";
    +    String endpointUrl = "https://kinesis.us-west-2.amazonaws.com";
    +    String region = "us-west-2";
    +    InitialPositionInStream initialPosition = InitialPositionInStream.TRIM_HORIZON;
    +    String appName = "a-very-nice-kinesis-app";
    +    Duration checkpointInterval = Seconds.apply(30);
    +    StorageLevel storageLevel = StorageLevel.MEMORY_ONLY();
    +
    +    KinesisInputDStream<byte[]> kinesisDStream = KinesisInputDStream.builder()
    +      .streamingContext(ssc)
    +      .streamName(streamName)
    +      .endpointUrl(endpointUrl)
    +      .regionName(region)
    +      .initialPositionInStream(initialPosition)
    +      .checkpointAppName(appName)
    +      .checkpointInterval(checkpointInterval)
    +      .storageLevel(storageLevel)
    +      .build();
    +    assert(kinesisDStream.streamName() == streamName);
    +    assert(kinesisDStream.endpointUrl() == endpointUrl);
    +    assert(kinesisDStream.regionName() == region);
    +    assert(kinesisDStream.initialPositionInStream() == initialPosition);
    +    assert(kinesisDStream.checkpointAppName() == appName);
    +    assert(kinesisDStream.checkpointInterval() == checkpointInterval);
    +    assert(kinesisDStream._storageLevel() == storageLevel);
    +    ssc.stop();
    --- End diff --
    
    Looks like this will need to be left as-is. In the current test implementation we check that  ```checkpointInterval``` isn't a required option and its default value is obtained via ```ssc.graph.batchDuration```, which we won't be able to mock because ```DStreamGraph``` is final.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107750451
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 org.apache.spark.streaming.TestSuiteBase
    +import org.apache.spark.util.Utils
    +
    +class SparkAWSCredentialsBuilderSuite extends TestSuiteBase {
    +  private def builder = SparkAWSCredentials.builder
    +
    +  private val basicCreds = BasicCredentials(
    +    awsAccessKeyId = "a-very-nice-access-key",
    +    awsSecretKey = "a-very-nice-secret-key")
    +
    +  private val stsCreds = STSCredentials(
    +    stsRoleArn = "a-very-nice-role-arn",
    +    stsSessionName = "a-very-nice-secret-key",
    +    stsExternalId = Option("a-very-nice-external-id"),
    +    longLivedCredsProvider = basicCreds)
    +
    +  test("should build DefaultCredentials when given no params") {
    +    assert(builder.build() == DefaultCredentials)
    +  }
    +
    +  test("should build BasicCredentials") {
    +    assertResult(basicCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +  }
    +
    +  test("should build STSCredentials") {
    +    // No external ID, default long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None,
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .build()
    +    }
    +    // Default long-lived creds
    +    assertResult(stsCreds.copy(
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .build()
    +    }
    +    // No external ID, basic keypair for long-lived creds
    +    assertResult(stsCreds.copy(
    --- End diff --
    
    nit: could you make this a single line? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Thanks! new PR would be easier!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz Actually, now that I think about it, do we need to make ```messageHandler``` a constructor arg since ```Builder``` is a generic class? There's probably a way we could get around this but I'd imagine it would be pretty complex...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74901 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74901/testReport)** for PR 17250 at commit [`d6afaef`](https://github.com/apache/spark/commit/d6afaef4099d9b20d0d1257ec5942d1bb5b868af).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107318994
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    +  /**
    +   * Builder for [[SerializableCredentialsProvider]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder {
    +    private var awsAccessKeyId: Option[String] = None
    +    private var awsSecretKey: Option[String] = None
    +    private var stsRoleArn: Option[String] = None
    +    private var stsSessionName: Option[String] = None
    +    private var stsExternalId: Option[String] = None
    +
    +
    +    /**
    +     * Sets the AWS access key ID when using a basic AWS keypair for long-lived authorization
    +     * credentials. A value must also be provided for the AWS secret key.
    +     *
    +     * @param accessKeyId AWS access key ID
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def awsAccessKeyId(accessKeyId: String): Builder = {
    --- End diff --
    
    I'll rework this builder to take multiple arguments for the long-lived keypair and STS


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Good point @budde. I can think of two options:
    
     1. Leave it as a constructor param
     2. Make the `Builder` class non-generic and have the `build` function take the message handler:
    
    ```scala
    class Builder {
      
      def build(): KinesisInputDStream[Array[Byte]]
    
      def buildWithMessageHandler[T](f: Record => T): KinesisInputDStream[T]
    }
    ```
    
    It's a matter of taking it as the first parameter or the final parameter. There are other ways to do it as well, but will throw runtime exceptions instead of at compile time.
    
    cc @rxin for input on APIs


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107748154
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,238 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      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 [[SerialziableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    --- End diff --
    
    ditto


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105791929
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    --- End diff --
    
    I would also make this a `required` field, otherwise people will face confusing issues when they start 2 streams from the same Spark application.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107296097
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    +object KinesisInputDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  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[SerializableCredentialsProvider] = None
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = 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 [[SerializableCredentialsProvider]] to use for authenticating to the AWS Kinesis
    +     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
    +     *
    +     * @param credentials [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    +     */
    +    def kinesisCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      kinesisCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    +     */
    +    def dynamoDBCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      dynamoDBCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    +     */
    +    def cloudWatchCredentials(credentials: SerializableCredentialsProvider): 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),
    +        handler,
    +        kinesisCredsProvider.getOrElse(DefaultCredentialsProvider),
    +        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")
    +    }
    +
    +    private def buildSTSCredentials(
    --- End diff --
    
    I guess you no longer need this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107025906
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    +object KinesisInputDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  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[SerializableCredentialsProvider] = None
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = 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 [[SerializableCredentialsProvider]] to use for authenticating to the AWS Kinesis
    +     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
    +     *
    +     * @param credentials [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    +     */
    +    def kinesisCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      kinesisCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    +     */
    +    def dynamoDBCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      dynamoDBCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    +     */
    +    def cloudWatchCredentials(credentials: SerializableCredentialsProvider): 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),
    +        handler,
    +        kinesisCredsProvider.getOrElse(DefaultCredentialsProvider),
    +        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")
    +    }
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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] = {
    --- End diff --
    
    How about keeping it here and refactoring ```KinesisUtils``` to use it? I think this is what I was intending to do originally, just forgot to update the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107765330
  
    --- Diff: external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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 org.junit.Test;
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
    +
    +import org.apache.spark.storage.StorageLevel;
    +import org.apache.spark.streaming.Duration;
    +import org.apache.spark.streaming.Seconds;
    +import org.apache.spark.streaming.LocalJavaStreamingContext;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +
    +public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext {
    +  /**
    +   * Basic test to ensure that the KinesisDStream.Builder interface is accessible from Java.
    +   */
    +  @Test
    +  public void testJavaKinesisDStreamBuilder() {
    +    String streamName = "a-very-nice-stream-name";
    +    String endpointUrl = "https://kinesis.us-west-2.amazonaws.com";
    +    String region = "us-west-2";
    +    InitialPositionInStream initialPosition = InitialPositionInStream.TRIM_HORIZON;
    +    String appName = "a-very-nice-kinesis-app";
    +    Duration checkpointInterval = Seconds.apply(30);
    +    StorageLevel storageLevel = StorageLevel.MEMORY_ONLY();
    +
    +    KinesisInputDStream<byte[]> kinesisDStream = KinesisInputDStream.builder()
    +      .streamingContext(ssc)
    +      .streamName(streamName)
    +      .endpointUrl(endpointUrl)
    +      .regionName(region)
    +      .initialPositionInStream(initialPosition)
    +      .checkpointAppName(appName)
    +      .checkpointInterval(checkpointInterval)
    +      .storageLevel(storageLevel)
    +      .build();
    +    assert(kinesisDStream.streamName() == streamName);
    +    assert(kinesisDStream.endpointUrl() == endpointUrl);
    +    assert(kinesisDStream.regionName() == region);
    +    assert(kinesisDStream.initialPositionInStream() == initialPosition);
    +    assert(kinesisDStream.checkpointAppName() == appName);
    +    assert(kinesisDStream.checkpointInterval() == checkpointInterval);
    +    assert(kinesisDStream._storageLevel() == storageLevel);
    +    ssc.stop();
    --- End diff --
    
    that's fine then. As long as we don't break the environment for other tests, do proper clean up, it should be fine


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74379/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107759746
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 org.apache.spark.streaming.TestSuiteBase
    +import org.apache.spark.util.Utils
    +
    +class SparkAWSCredentialsBuilderSuite extends TestSuiteBase {
    +  private def builder = SparkAWSCredentials.builder
    +
    +  private val basicCreds = BasicCredentials(
    +    awsAccessKeyId = "a-very-nice-access-key",
    +    awsSecretKey = "a-very-nice-secret-key")
    +
    +  private val stsCreds = STSCredentials(
    +    stsRoleArn = "a-very-nice-role-arn",
    +    stsSessionName = "a-very-nice-secret-key",
    +    stsExternalId = Option("a-very-nice-external-id"),
    +    longLivedCredsProvider = basicCreds)
    +
    +  test("should build DefaultCredentials when given no params") {
    +    assert(builder.build() == DefaultCredentials)
    +  }
    +
    +  test("should build BasicCredentials") {
    +    assertResult(basicCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +  }
    +
    +  test("should build STSCredentials") {
    +    // No external ID, default long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None,
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .build()
    +    }
    +    // Default long-lived creds
    +    assertResult(stsCreds.copy(
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .build()
    +    }
    +    // No external ID, basic keypair for long-lived creds
    +    assertResult(stsCreds.copy(
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75172 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75172/testReport)** for PR 17250 at commit [`03f91da`](https://github.com/apache/spark/commit/03f91dadb3878aa47f2a134e3e1b8d46aadd3b47).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Thanks a lot for the quick turnaround @budde ! Could you also contribute to the docs as well with the new builder API?
    https://spark.apache.org/docs/latest/streaming-kinesis-integration.html


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz Updated per your feedback, thanks for taking a thorough look. I also renamed the ```longLivedCredsProvider``` of ```STSCredentials``` to just ```longLivedCreds``` to match the updated naming conventions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75170 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75170/testReport)** for PR 17250 at commit [`5315f1e`](https://github.com/apache/spark/commit/5315f1e9f200be7f97d7838c2d64bc6bfb208ecb).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74901 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74901/testReport)** for PR 17250 at commit [`d6afaef`](https://github.com/apache/spark/commit/d6afaef4099d9b20d0d1257ec5942d1bb5b868af).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class Builder `
      * `  class Builder `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107759395
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 java.lang.IllegalArgumentException
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Seconds, StreamingContext, TestSuiteBase}
    +
    +class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterEach
    +   with MockitoSugar {
    +  import KinesisInputDStream._
    +
    +  private val ssc = new StreamingContext(conf, batchDuration)
    +  private val streamName = "a-very-nice-kinesis-stream-name"
    +  private val checkpointAppName = "a-very-nice-kcl-app-name"
    +  private def baseBuilder = KinesisInputDStream.builder
    +  private def builder = baseBuilder.streamingContext(ssc)
    +    .streamName(streamName)
    +    .checkpointAppName(checkpointAppName)
    +
    +  override def afterAll(): Unit = {
    +    ssc.stop()
    +  }
    +
    +  test("should raise an exception if any required field is missing") {
    --- End diff --
    
    Sounds good


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806339
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    --- End diff --
    
    Yeah, I mostly put this here for the sake of completeness. Should be able to remove this if we introduce a credential builder and replace it with a single method that just takes a ```SerializableCredentialsProvider``` instance to use for the service. A user could always just manually pass ```DefaultCredentialsProvider``` if they really want to for some reason.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74909 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74909/testReport)** for PR 17250 at commit [`3cc2df8`](https://github.com/apache/spark/commit/3cc2df82f5ae6b4e3a21c2549a8855f5bd8a3eb6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  class Builder `
      * `  class Builder `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806326
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    --- End diff --
    
    I'll expand the builder to recognize required params and remove them as ctor args


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107759010
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,238 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      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 [[SerialziableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    --- End diff --
    
    Will fix and do a "grep -r 'SerialziableCredentialsProvider' *" to make sure this isn't appearing anywhere else


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75170/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107318916
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    --- End diff --
    
    I agree we should definitely come up with a better name here. What about ```SparkAWSCredentials```? Obviously it's not as succinct as ```AWSCredentials``` but I think it's a clear name that avoids collisions.
    
    I'm okay with ```CredentialsProvider``` otherwise.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806348
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    --- End diff --
    
    We'll make this requited


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107295179
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    +  /**
    +   * Builder for [[SerializableCredentialsProvider]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder {
    +    private var awsAccessKeyId: Option[String] = None
    +    private var awsSecretKey: Option[String] = None
    +    private var stsRoleArn: Option[String] = None
    +    private var stsSessionName: Option[String] = None
    +    private var stsExternalId: Option[String] = None
    +
    +
    +    /**
    +     * Sets the AWS access key ID when using a basic AWS keypair for long-lived authorization
    +     * credentials. A value must also be provided for the AWS secret key.
    +     *
    +     * @param accessKeyId AWS access key ID
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def awsAccessKeyId(accessKeyId: String): Builder = {
    +      awsAccessKeyId = Option(accessKeyId)
    +      this
    +    }
    +
    +    /**
    +     * Sets the AWS secret key when using a basic AWS keypair for long-lived authorization
    +     * credentials. A value must also be provided for the AWS access key ID.
    +     *
    +     * @param secretKey AWS secret key
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def awsSecretKey(secretKey: String): Builder = {
    +      awsSecretKey = Option(secretKey)
    +      this
    +    }
    +
    +    /**
    +     * Sets the ARN of the IAM role to assume when using AWS STS for temporary session-based
    +     * authentication. A value must also be provided for the STS session name.
    +     *
    +     * @param roleArn ARN of IAM role to assume via STS
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def stsRoleArn(roleArn: String): Builder = {
    --- End diff --
    
    same here: 
    ```scala
    def withSts(roleArn: String, sessionName: String)
    def withSts(roleArn: String, sessionName: String, externalId: String)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105792289
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * Sets the AWS Kinesis endpoint URL. Defaults to "https://kinesis.us-east-1.amazonaws.com" if
    +     * no custom value is specified
    --- End diff --
    
    Would be great to document what happens when both region and endpoint is set, but are referring to different regions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz Sure, want me to add it to this PR or open a new one?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merging to master


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107296327
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    --- End diff --
    
    ditto, let's call it evolving for now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105791711
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    --- End diff --
    
    I feel that builders rarely have constructor params. I understand you wanted to make these the `required` options, but I would just rather have the builder have a `zero-param` constructor, and it checks for the required fields upon `build()`. What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107330754
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    --- End diff --
    
    I guess `SparkAWSCredentials` also work 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #74909 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74909/testReport)** for PR 17250 at commit [`3cc2df8`](https://github.com/apache/spark/commit/3cc2df82f5ae6b4e3a21c2549a8855f5bd8a3eb6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107762239
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -22,24 +22,28 @@ 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,
    -    streamName: String,
    -    endpointUrl: String,
    -    regionName: String,
    -    initialPositionInStream: InitialPositionInStream,
    -    checkpointAppName: String,
    -    checkpointInterval: Duration,
    -    storageLevel: StorageLevel,
    -    messageHandler: Record => T,
    -    kinesisCredsProvider: SerializableCredentialsProvider
    +    private[kinesis] val streamName: String,
    --- End diff --
    
    Yeah, was just thinking that'd be a lot simpler. I'll go that route. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107761342
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,238 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      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 [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    --- End diff --
    
    Figured out why I didn't catch this before-- apparently I can't spell "serializable". Ugh!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz Thanks for taking a look!
    
    Re: major feedback:
    
    1. Sounds reasonable to me. I don't have strong feeling here.
    1. Will do.
    1. Sounds good.
    1. I agree that method overloading isn't a great and could lead to a similar scenario as ```KinesisUtils.createStream()``` in the long run. I kept it here since I figured that the scope was rather limited and I wanted to avoid having to deal with too many builders but I think you raise a good point. I'll look into adding a credentials builder. Ideally we would only have to worry about Java/Python interoperability and could just use a single method optionals or default args to solve this but I guess we have to play the hands we're dealt :)
    
    I can move the builder to the companion object of ```KinesisInputDStream``` (this is actually where I  had this placed originally). I like the idea of the private constructor as well although I don't think this will be possible until ```KinesisUtils.createStream()``` is refactored to use the builder pattern.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806315
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    +    private var checkpointInterval = ssc.graph.batchDuration
    +    private var storageLevel = DEFAULT_STORAGE_LEVEL
    +    private var kinesisCredsProvider: SerializableCredentialsProvider = DefaultCredentialsProvider
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = None
    +
    +    /**
    +     * 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 [[KinesisDStream.Builder]]
    +     */
    +    def endpointUrl(url: String): Builder[T] = {
    +      endpointUrl = 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 [[KinesisDStream.Builder]]
    +     */
    +    def regionName(regionName: String): Builder[T] = {
    +      this.regionName = 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 [[KinesisDStream.Builder]]
    +     */
    +    def initialPositionInStream(initialPosition: InitialPositionInStream): Builder[T] = {
    +      initialPositionInStream = initialPosition
    +      this
    +    }
    +
    +    /**
    +     * Sets the KCL application name to use when checkpointing state to DynamoDB. Defaults to the
    +     * Spark app name if no custom value is specified.
    +     *
    +     * @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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointAppName(appName: String): Builder[T] = {
    +      checkpointAppName = appName
    +      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 [[KinesisDStream.Builder]]
    +     */
    +    def checkpointInterval(interval: Duration): Builder[T] = {
    +      checkpointInterval = 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 [[KinesisDStream.Builder]]
    +     */
    +    def storageLevel(storageLevel: StorageLevel): Builder[T] = {
    +      this.storageLevel = storageLevel
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the Kinesis service. This
    +     * is also the default credentials provider used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisDefaultCredentials(): Builder[T] = {
    +      kinesisCredsProvider = DefaultCredentialsProvider
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the Kinesis service. The default AWS
    +     * credentials provider chain is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(stsRoleArn, stsSessionName)
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the Kinesis service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def kinesisSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      kinesisCredsProvider = buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the DynamoDB service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBDefaultCredentials(): Builder[T] = {
    +      dynamoDBCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the DynamoDB service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the DynamoDB service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def dynamoDBSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      dynamoDBCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Use the default AWS credentials provider chain to authenticate to the CloudWatch service. The
    +     * Kinesis credentials provider is used if no custom value is specified.
    +     *
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchDefaultCredentials(): Builder[T] = {
    +      cloudWatchCredsProvider = Option(DefaultCredentialsProvider)
    +      this
    +    }
    +
    +    /**
    +     * Use an AWS credential keypair to authenticate to the CloudWatch service. The Kinesis
    +     * credentials provider is used if no custom value is specified.
    +     *
    +     * @param awsAccessKeyId AWS access key ID
    +     * @param awsSecretKey AWS secret key
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchBasicAWSCredentials(awsAccessKeyId: String, awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(BasicCredentialsProvider(
    +        awsAccessKeyId = awsAccessKeyId,
    +        awsSecretKey = awsSecretKey))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(stsRoleArn: String, stsSessionName: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(stsRoleArn, stsSessionName))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and no external ID is used.
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The default
    +     * credential chain is used to authenticate to STS and the provided external ID is
    +     * validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId)))
    +      this
    +    }
    +
    +    /**
    +     * Assume an STS role in order to authenticate to the CloudWatch service. The provided
    +     * AWS credential keypair is used to authenticate to STS and the provided external ID
    +     * is validated against the value in the assumed role's policy (if one is specified).
    +     *
    +     * @param stsRoleArn ARN of the IAM role to assume via STS
    +     * @param stsSessionName Name to identify STS sessions by
    +     * @param awsAccessKeyId AWS access key ID for authenicating to STS
    +     * @param stsExternaId External ID used by STS to validate IAM role trust policy
    +     * @param awsSecretKey AWS secret key for authenticating to STS
    +     * @return Reference to this [[KinesisDStream.Builder]]
    +     */
    +    def cloudWatchSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: String,
    +        awsAccessKeyId: String,
    +        awsSecretKey: String): Builder[T] = {
    +      cloudWatchCredsProvider = Option(buildSTSCredentials(
    +        stsRoleArn,
    +        stsSessionName,
    +        stsExternalId = Option(stsExternalId),
    +        awsKeypair = Option(awsAccessKeyId, awsSecretKey)))
    +      this
    +    }
    +
    +    /**
    +     * Create a new instance of [[KinesisInputDStream]] with configured parameters.
    +     *
    +     * @return Instance of [[KinesisInputDStream]] constructed with configured parameters
    +     */
    +    def build(): KinesisInputDStream[T] = new KinesisInputDStream(
    +      ssc,
    +      streamName,
    +      endpointUrl,
    +      regionName,
    +      initialPositionInStream,
    +      checkpointAppName,
    +      checkpointInterval,
    +      storageLevel,
    +      messageHandler,
    +      kinesisCredsProvider,
    +      dynamoDBCredsProvider,
    +      cloudWatchCredsProvider)
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] =
    +    new Builder(ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param ssc StreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(ssc: StreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(ssc, streamName, KinesisUtils.defaultMessageHandler(_))
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[T]] using the provided message handler.
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @param messageHandler Handler used to extract type [[T]] from a Kinesis [[Record]]
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder[T: ClassTag](
    +      jssc: JavaStreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T): Builder[T] = builder(jssc.ssc, streamName, messageHandler)
    +
    +  /**
    +   * Creates a [[KinesisDStream.Builder]] for constructing [[KinesisInputDStream]] instances
    +   * of type [[Array[Byte]].
    +   *
    +   * @since 2.2.0
    +   *
    +   * @param jssc JavaStreamingContext to construct DStream with
    +   * @param streamName Name of Kinesis stream to read from
    +   * @return [[KinesisDStream.Builder]] instance
    +   */
    +  def builder(jssc: JavaStreamingContext, streamName: String): Builder[Array[Byte]]
    +    = builder(jssc.ssc, streamName)
    +
    +  private[kinesis] val DEFAULT_KINESIS_ENDPOINT_URL: String
    +    = "https://kinesis.us-east-1.amazonaws.com"
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    **[Test build #75065 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75065/testReport)** for PR 17250 at commit [`337b6ba`](https://github.com/apache/spark/commit/337b6ba575c969417566ee65664c93916e6923d4).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107749941
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 java.lang.IllegalArgumentException
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Seconds, StreamingContext, TestSuiteBase}
    +
    +class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterEach
    +   with MockitoSugar {
    +  import KinesisInputDStream._
    +
    +  private val ssc = new StreamingContext(conf, batchDuration)
    +  private val streamName = "a-very-nice-kinesis-stream-name"
    +  private val checkpointAppName = "a-very-nice-kcl-app-name"
    +  private def baseBuilder = KinesisInputDStream.builder
    +  private def builder = baseBuilder.streamingContext(ssc)
    +    .streamName(streamName)
    +    .checkpointAppName(checkpointAppName)
    +
    +  override def afterAll(): Unit = {
    +    ssc.stop()
    +  }
    +
    +  test("should raise an exception if any required field is missing") {
    --- End diff --
    
    I would split this into 3 very small tests:
    `should raise an exception if StreamingContext is missing`
    `should raise an exception if stream name is missing`
    `should raise an exception if checkpoint app name is missing`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107968898
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * 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 com.amazonaws.auth._
    +
    +import org.apache.spark.annotation.InterfaceStability
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * Serializable interface providing a method executors can call to obtain an
    + * AWSCredentialsProvider instance for authenticating to AWS services.
    + */
    +private[kinesis] sealed trait SparkAWSCredentials extends Serializable {
    +  /**
    +   * Return an AWSCredentialProvider instance that can be used by the Kinesis Client
    +   * Library to authenticate to AWS services (Kinesis, CloudWatch and DynamoDB).
    +   */
    +  def provider: AWSCredentialsProvider
    +}
    +
    +/** Returns DefaultAWSCredentialsProviderChain for authentication. */
    +private[kinesis] final case object DefaultCredentials extends SparkAWSCredentials {
    +
    +  def provider: AWSCredentialsProvider = new DefaultAWSCredentialsProviderChain
    +}
    +
    +/**
    + * Returns AWSStaticCredentialsProvider constructed using basic AWS keypair. Falls back to using
    + * DefaultCredentialsProviderChain if unable to construct a AWSCredentialsProviderChain
    + * instance with the provided arguments (e.g. if they are null).
    + */
    +private[kinesis] final case class BasicCredentials(
    +    awsAccessKeyId: String,
    +    awsSecretKey: String) extends SparkAWSCredentials with Logging {
    +
    +  def provider: AWSCredentialsProvider = try {
    +    new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKeyId, awsSecretKey))
    +  } catch {
    +    case e: IllegalArgumentException =>
    +      logWarning("Unable to construct AWSStaticCredentialsProvider with provided keypair; " +
    +        "falling back to DefaultCredentialsProviderChain.", e)
    +      new DefaultAWSCredentialsProviderChain
    +  }
    +}
    +
    +/**
    + * Returns an STSAssumeRoleSessionCredentialsProvider instance which assumes an IAM
    + * role in order to authenticate against resources in an external account.
    + */
    +private[kinesis] final case class STSCredentials(
    +    stsRoleArn: String,
    +    stsSessionName: String,
    +    stsExternalId: Option[String] = None,
    +    longLivedCreds: SparkAWSCredentials = DefaultCredentials)
    +  extends SparkAWSCredentials  {
    +
    +  def provider: AWSCredentialsProvider = {
    +    val builder = new STSAssumeRoleSessionCredentialsProvider.Builder(stsRoleArn, stsSessionName)
    +      .withLongLivedCredentialsProvider(longLivedCreds.provider)
    +    stsExternalId match {
    +      case Some(stsExternalId) =>
    +        builder.withExternalId(stsExternalId)
    +          .build()
    +      case None =>
    +        builder.build()
    +    }
    +  }
    +}
    +
    +@InterfaceStability.Evolving
    +object SparkAWSCredentials {
    +  /**
    +   * Builder for [[SparkAWSCredentials]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Evolving
    +  class Builder {
    +    private var basicCreds: Option[BasicCredentials] = None
    +    private var stsCreds: Option[STSCredentials] = None
    +
    +    /**
    +     * Use a basic AWS keypair for long-lived authorization.
    --- End diff --
    
    Do you want to add the note here as well?
    ```
    * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
    * is enabled. Make sure that your checkpoint directory is secure.
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107297624
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    --- End diff --
    
    How about simply `CredentialProvider`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r105806375
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisDStream.scala ---
    @@ -0,0 +1,559 @@
    +/*
    + * 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.storage.StorageLevel
    +import org.apache.spark.streaming.{Duration, StreamingContext}
    +import org.apache.spark.streaming.api.java.JavaStreamingContext
    +
    +@InterfaceStability.Stable
    +object KinesisDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @param ssc StreamingContext to be used for creating DStreams
    +   * @param streamName Name of Kinesis stream
    +   * @param messageHandler Function for extracting DStream data type from KCL Record
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder[T: ClassTag](
    +      ssc: StreamingContext,
    +      streamName: String,
    +      messageHandler: Record => T) {
    +    private var endpointUrl = DEFAULT_KINESIS_ENDPOINT_URL
    +    private var regionName = DEFAULT_KINESIS_REGION_NAME
    +    private var initialPositionInStream = DEFAULT_INITIAL_POSITION_IN_STREAM
    +    private var checkpointAppName = ssc.sparkContext.appName
    --- End diff --
    
    Sounds reasonable. Feel free to push back on any other defaults as well-- figured these would just be a starting point. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    @brkyvz PR has been updated, apologies for the delay. I've added ```SerializableCredentialsProvider.Builder```, which I'm willing to hear suggestions for a better name on. I wanted to stay away from something like ```AWSCredentials.Builder``` so as to avoid confusion with similarly-named classes in the AWS Java SDK.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107973850
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 com.amazonaws.auth._
    +
    +import org.apache.spark.annotation.InterfaceStability
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * Serializable interface providing a method executors can call to obtain an
    + * AWSCredentialsProvider instance for authenticating to AWS services.
    + */
    +private[kinesis] sealed trait SparkAWSCredentials extends Serializable {
    +  /**
    +   * Return an AWSCredentialProvider instance that can be used by the Kinesis Client
    +   * Library to authenticate to AWS services (Kinesis, CloudWatch and DynamoDB).
    +   */
    +  def provider: AWSCredentialsProvider
    +}
    +
    +/** Returns DefaultAWSCredentialsProviderChain for authentication. */
    +private[kinesis] final case object DefaultCredentials extends SparkAWSCredentials {
    +
    +  def provider: AWSCredentialsProvider = new DefaultAWSCredentialsProviderChain
    +}
    +
    +/**
    + * Returns AWSStaticCredentialsProvider constructed using basic AWS keypair. Falls back to using
    + * DefaultCredentialsProviderChain if unable to construct a AWSCredentialsProviderChain
    + * instance with the provided arguments (e.g. if they are null).
    + */
    +private[kinesis] final case class BasicCredentials(
    +    awsAccessKeyId: String,
    +    awsSecretKey: String) extends SparkAWSCredentials with Logging {
    +
    +  def provider: AWSCredentialsProvider = try {
    +    new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKeyId, awsSecretKey))
    +  } catch {
    +    case e: IllegalArgumentException =>
    +      logWarning("Unable to construct AWSStaticCredentialsProvider with provided keypair; " +
    +        "falling back to DefaultCredentialsProviderChain.", e)
    +      new DefaultAWSCredentialsProviderChain
    +  }
    +}
    +
    +/**
    + * Returns an STSAssumeRoleSessionCredentialsProvider instance which assumes an IAM
    + * role in order to authenticate against resources in an external account.
    + */
    +private[kinesis] final case class STSCredentials(
    +    stsRoleArn: String,
    +    stsSessionName: String,
    +    stsExternalId: Option[String] = None,
    +    longLivedCreds: SparkAWSCredentials = DefaultCredentials)
    +  extends SparkAWSCredentials  {
    +
    +  def provider: AWSCredentialsProvider = {
    +    val builder = new STSAssumeRoleSessionCredentialsProvider.Builder(stsRoleArn, stsSessionName)
    +      .withLongLivedCredentialsProvider(longLivedCreds.provider)
    +    stsExternalId match {
    +      case Some(stsExternalId) =>
    +        builder.withExternalId(stsExternalId)
    +          .build()
    +      case None =>
    +        builder.build()
    +    }
    +  }
    +}
    +
    +@InterfaceStability.Evolving
    +object SparkAWSCredentials {
    +  /**
    +   * Builder for [[SparkAWSCredentials]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Evolving
    +  class Builder {
    +    private var basicCreds: Option[BasicCredentials] = None
    +    private var stsCreds: Option[STSCredentials] = None
    +
    +    /**
    +     * Use a basic AWS keypair for long-lived authorization.
    +     *
    +     * @note The given AWS keypair will be saved in DStream checkpoints if checkpointing is
    +     * enabled. Make sure that your checkpoint directory is secure if using the default
    --- End diff --
    
    nit: `Make sure that your checkpoint directory is secure. Prefer using the [https://link.to.amazon.docs default credential provider chain]] if possible`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Forgot to stop the ```StreamingContext``` added in ```KinesisDStreamBuilderSuite```. Updated the code to stop the context after all tests have run.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107319004
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SerializableCredentialsProvider.scala ---
    @@ -83,3 +84,146 @@ private[kinesis] final case class STSCredentialsProvider(
         }
       }
     }
    +
    +@InterfaceStability.Stable
    +object SerializableCredentialsProvider {
    +  /**
    +   * Builder for [[SerializableCredentialsProvider]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  class Builder {
    +    private var awsAccessKeyId: Option[String] = None
    +    private var awsSecretKey: Option[String] = None
    +    private var stsRoleArn: Option[String] = None
    +    private var stsSessionName: Option[String] = None
    +    private var stsExternalId: Option[String] = None
    +
    +
    +    /**
    +     * Sets the AWS access key ID when using a basic AWS keypair for long-lived authorization
    +     * credentials. A value must also be provided for the AWS secret key.
    +     *
    +     * @param accessKeyId AWS access key ID
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def awsAccessKeyId(accessKeyId: String): Builder = {
    +      awsAccessKeyId = Option(accessKeyId)
    +      this
    +    }
    +
    +    /**
    +     * Sets the AWS secret key when using a basic AWS keypair for long-lived authorization
    +     * credentials. A value must also be provided for the AWS access key ID.
    +     *
    +     * @param secretKey AWS secret key
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def awsSecretKey(secretKey: String): Builder = {
    +      awsSecretKey = Option(secretKey)
    +      this
    +    }
    +
    +    /**
    +     * Sets the ARN of the IAM role to assume when using AWS STS for temporary session-based
    +     * authentication. A value must also be provided for the STS session name.
    +     *
    +     * @param roleArn ARN of IAM role to assume via STS
    +     * @return Reference to this [[SerializableCredentialsProvider.Builder]]
    +     */
    +    def stsRoleArn(roleArn: String): Builder = {
    --- End diff --
    
    Will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74418/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107003143
  
    --- Diff: external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala ---
    @@ -71,7 +75,256 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
     
       override def getReceiver(): Receiver[T] = {
         new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream,
    -      checkpointAppName, checkpointInterval, storageLevel, messageHandler,
    -      kinesisCredsProvider)
    +      checkpointAppName, checkpointInterval, _storageLevel, messageHandler,
    +      kinesisCredsProvider, dynamoDBCredsProvider, cloudWatchCredsProvider)
       }
     }
    +
    +@InterfaceStability.Stable
    +object KinesisInputDStream {
    +  /**
    +   * Builder for [[KinesisInputDStream]] instances.
    +   *
    +   * @since 2.2.0
    +   */
    +  @InterfaceStability.Stable
    +  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[SerializableCredentialsProvider] = None
    +    private var dynamoDBCredsProvider: Option[SerializableCredentialsProvider] = None
    +    private var cloudWatchCredsProvider: Option[SerializableCredentialsProvider] = 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 [[SerializableCredentialsProvider]] to use for authenticating to the AWS Kinesis
    +     * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified.
    +     *
    +     * @param credentials [[SerialziableCredentialsProvider]] to use for Kinesis authentication
    +     */
    +    def kinesisCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      kinesisCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for DynamoDB authentication
    +     */
    +    def dynamoDBCredentials(credentials: SerializableCredentialsProvider): Builder = {
    +      dynamoDBCredsProvider = Option(credentials)
    +      this
    +    }
    +
    +    /**
    +     * Sets the [[SerializableCredentialsProvider]] 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 [[SerialziableCredentialsProvider]] to use for CloudWatch authentication
    +     */
    +    def cloudWatchCredentials(credentials: SerializableCredentialsProvider): 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),
    +        handler,
    +        kinesisCredsProvider.getOrElse(DefaultCredentialsProvider),
    +        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")
    +    }
    +
    +    private def buildSTSCredentials(
    +        stsRoleArn: String,
    +        stsSessionName: String,
    +        stsExternalId: Option[String] = None,
    +        awsKeypair: Option[(String, String)] = None): STSCredentialsProvider = {
    +      val baseCredentials = STSCredentialsProvider(
    +        stsRoleArn = stsRoleArn,
    +        stsSessionName = stsSessionName,
    +        stsExternalId = stsExternalId)
    +      awsKeypair match {
    +        case Some((accessKey, secretKey)) =>
    +          baseCredentials.copy(
    +            longLivedCredsProvider = BasicCredentialsProvider(accessKey, secretKey))
    +        case None =>
    +          baseCredentials
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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] = {
    --- End diff --
    
    would you mind just re-using the code in `KinesisUtils` instead of copying the code?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74351/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107750889
  
    --- Diff: external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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 org.apache.spark.streaming.TestSuiteBase
    +import org.apache.spark.util.Utils
    +
    +class SparkAWSCredentialsBuilderSuite extends TestSuiteBase {
    +  private def builder = SparkAWSCredentials.builder
    +
    +  private val basicCreds = BasicCredentials(
    +    awsAccessKeyId = "a-very-nice-access-key",
    +    awsSecretKey = "a-very-nice-secret-key")
    +
    +  private val stsCreds = STSCredentials(
    +    stsRoleArn = "a-very-nice-role-arn",
    +    stsSessionName = "a-very-nice-secret-key",
    +    stsExternalId = Option("a-very-nice-external-id"),
    +    longLivedCredsProvider = basicCreds)
    +
    +  test("should build DefaultCredentials when given no params") {
    +    assert(builder.build() == DefaultCredentials)
    +  }
    +
    +  test("should build BasicCredentials") {
    +    assertResult(basicCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +  }
    +
    +  test("should build STSCredentials") {
    +    // No external ID, default long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None,
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .build()
    +    }
    +    // Default long-lived creds
    +    assertResult(stsCreds.copy(
    +      longLivedCredsProvider = DefaultCredentials)) {
    +      builder.stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .build()
    +    }
    +    // No external ID, basic keypair for long-lived creds
    +    assertResult(stsCreds.copy(
    +      stsExternalId = None)) {
    +      builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName)
    +        .basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +    // Basic keypair for long-lived creds
    +    assertResult(stsCreds) {
    +      builder.stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .build()
    +    }
    +    // Order shouldn't matter
    +    assertResult(stsCreds) {
    +      builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey)
    +        .stsCredentials(
    +          stsCreds.stsRoleArn,
    +          stsCreds.stsSessionName,
    +          stsCreds.stsExternalId.get)
    +        .build()
    +    }
    +  }
    +
    +  test("SparkAWSCredentials classes should be serializable") {
    +    Utils.deserialize[BasicCredentials](
    --- End diff --
    
    I know we didn't have it before, but could you also check that after deserialization, they're equivalent?
    ```scala
    val creds = BasicCredentials("x", "y")
    assert(Utils.deserialize[BasicCredentials](Utils.serialize(creds)) === creds)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #17250: [SPARK-19911][STREAMING] Add builder interface fo...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17250#discussion_r107749649
  
    --- Diff: external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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 org.junit.Test;
    +
    +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
    +
    +import org.apache.spark.storage.StorageLevel;
    +import org.apache.spark.streaming.Duration;
    +import org.apache.spark.streaming.Seconds;
    +import org.apache.spark.streaming.LocalJavaStreamingContext;
    +import org.apache.spark.streaming.api.java.JavaDStream;
    +
    +public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext {
    +  /**
    +   * Basic test to ensure that the KinesisDStream.Builder interface is accessible from Java.
    +   */
    +  @Test
    +  public void testJavaKinesisDStreamBuilder() {
    +    String streamName = "a-very-nice-stream-name";
    +    String endpointUrl = "https://kinesis.us-west-2.amazonaws.com";
    +    String region = "us-west-2";
    +    InitialPositionInStream initialPosition = InitialPositionInStream.TRIM_HORIZON;
    +    String appName = "a-very-nice-kinesis-app";
    +    Duration checkpointInterval = Seconds.apply(30);
    +    StorageLevel storageLevel = StorageLevel.MEMORY_ONLY();
    +
    +    KinesisInputDStream<byte[]> kinesisDStream = KinesisInputDStream.builder()
    +      .streamingContext(ssc)
    +      .streamName(streamName)
    +      .endpointUrl(endpointUrl)
    +      .regionName(region)
    +      .initialPositionInStream(initialPosition)
    +      .checkpointAppName(appName)
    +      .checkpointInterval(checkpointInterval)
    +      .storageLevel(storageLevel)
    +      .build();
    +    assert(kinesisDStream.streamName() == streamName);
    +    assert(kinesisDStream.endpointUrl() == endpointUrl);
    +    assert(kinesisDStream.regionName() == region);
    +    assert(kinesisDStream.initialPositionInStream() == initialPosition);
    +    assert(kinesisDStream.checkpointAppName() == appName);
    +    assert(kinesisDStream.checkpointInterval() == checkpointInterval);
    +    assert(kinesisDStream._storageLevel() == storageLevel);
    +    ssc.stop();
    --- End diff --
    
    where does this come from? `LocalJavaStreamingContext`? If so, I wouldn't stop it. In fact, you can use `Mockito` to create a mock `JavaStreamingContext` if you like


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #17250: [SPARK-19911][STREAMING] Add builder interface for Kines...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/17250
  
    Open questions I'd like feedback on:
    
    * Should the ```KinesisUtils.createStream()``` methods be marked as deprecated?
    * Should the ```KinesisUtils.createStream()``` methods be refactored to use the builder?
    * Should I add full docs for each method (e.g. including ```@param``` lists)?
    * Does the file name and class name I've added seem reasonable?
    * Is making the ctor args to ```KinesisInputDStream``` package private for testing reasonable?
    
    I'd like to also extend this to allow configuring CloudWatch and DynamoDB-specific authorization which I imagine will be quite helpful to users. I'm trying to decide if I should do this as a separate PR or just roll it in here.
    
    Pinging @brkyvz and @srown from #16744


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org