-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-27420][DSTREAMS][Kinesis] KinesisInputDStream should expose a way to configure CloudWatch metrics #24651
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 4 commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
c4ada94
[SPARK-27420][DSTREAMS][Kinesis] KinesisInputDStream should expose a …
sekikn a7800e6
Address scalastyle errors
sekikn fd2229c
Fix a wrong scalastyle directive
sekikn d487679
Merge remote-tracking branch 'upstream/master' into SPARK-27420
sekikn 90e4d90
Merge remote-tracking branch 'upstream/master' into SPARK-27420
sekikn c9341a0
[SPARK-27420] Additional documentation fixes
sekikn File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,9 @@ package org.apache.spark.streaming.kinesis | |
|
|
||
| import scala.reflect.ClassTag | ||
|
|
||
| import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream | ||
| import collection.JavaConverters._ | ||
| import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration} | ||
| import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel | ||
| import com.amazonaws.services.kinesis.model.Record | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
|
|
@@ -43,7 +45,9 @@ private[kinesis] class KinesisInputDStream[T: ClassTag]( | |
| val messageHandler: Record => T, | ||
| val kinesisCreds: SparkAWSCredentials, | ||
| val dynamoDBCreds: Option[SparkAWSCredentials], | ||
| val cloudWatchCreds: Option[SparkAWSCredentials] | ||
| val cloudWatchCreds: Option[SparkAWSCredentials], | ||
| val metricsLevel: MetricsLevel, | ||
| val metricsEnabledDimensions: Set[String] | ||
| ) extends ReceiverInputDStream[T](_ssc) { | ||
|
|
||
| import KinesisReadConfigurations._ | ||
|
|
@@ -79,7 +83,8 @@ private[kinesis] class KinesisInputDStream[T: ClassTag]( | |
| override def getReceiver(): Receiver[T] = { | ||
| new KinesisReceiver(streamName, endpointUrl, regionName, initialPosition, | ||
| checkpointAppName, checkpointInterval, _storageLevel, messageHandler, | ||
| kinesisCreds, dynamoDBCreds, cloudWatchCreds) | ||
| kinesisCreds, dynamoDBCreds, cloudWatchCreds, | ||
| metricsLevel, metricsEnabledDimensions) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -104,6 +109,8 @@ object KinesisInputDStream { | |
| private var kinesisCredsProvider: Option[SparkAWSCredentials] = None | ||
| private var dynamoDBCredsProvider: Option[SparkAWSCredentials] = None | ||
| private var cloudWatchCredsProvider: Option[SparkAWSCredentials] = None | ||
| private var metricsLevel: Option[MetricsLevel] = None | ||
| private var metricsEnabledDimensions: Option[Set[String]] = None | ||
|
|
||
| /** | ||
| * Sets the StreamingContext that will be used to construct the Kinesis DStream. This is a | ||
|
|
@@ -237,6 +244,7 @@ object KinesisInputDStream { | |
| * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified. | ||
| * | ||
| * @param credentials [[SparkAWSCredentials]] to use for Kinesis authentication | ||
| * @return Reference to this [[KinesisInputDStream.Builder]] | ||
| */ | ||
| def kinesisCredentials(credentials: SparkAWSCredentials): Builder = { | ||
| kinesisCredsProvider = Option(credentials) | ||
|
|
@@ -248,6 +256,7 @@ object KinesisInputDStream { | |
| * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set. | ||
| * | ||
| * @param credentials [[SparkAWSCredentials]] to use for DynamoDB authentication | ||
| * @return Reference to this [[KinesisInputDStream.Builder]] | ||
| */ | ||
| def dynamoDBCredentials(credentials: SparkAWSCredentials): Builder = { | ||
| dynamoDBCredsProvider = Option(credentials) | ||
|
|
@@ -259,12 +268,45 @@ object KinesisInputDStream { | |
| * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set. | ||
| * | ||
| * @param credentials [[SparkAWSCredentials]] to use for CloudWatch authentication | ||
| * @return Reference to this [[KinesisInputDStream.Builder]] | ||
| */ | ||
| def cloudWatchCredentials(credentials: SparkAWSCredentials): Builder = { | ||
| cloudWatchCredsProvider = Option(credentials) | ||
| this | ||
| } | ||
|
|
||
| // scalastyle:off line.size.limit | ||
| /** | ||
| * Sets the CloudWatch metrics level. Defaults to | ||
| * [[KinesisClientLibConfiguration.DEFAULT_METRICS_LEVEL]] if no custom value is specified. | ||
| * | ||
| * @param metricsLevel [[MetricsLevel]] to specify the CloudWatch metrics level | ||
| * @return Reference to this [[KinesisInputDStream.Builder]] | ||
| * @see [[https://docs.aws.amazon.com/streams/latest/dev/monitoring-with-kcl.html#metric-levels]] | ||
| */ | ||
| // scalastyle:on line.size.limit | ||
| def metricsLevel(metricsLevel: MetricsLevel): Builder = { | ||
| this.metricsLevel = Option(metricsLevel) | ||
| this | ||
| } | ||
|
|
||
| // scalastyle:off line.size.limit | ||
| /** | ||
| * Sets the enabled CloudWatch metrics dimensions. Defaults to | ||
| * [[KinesisClientLibConfiguration.DEFAULT_METRICS_ENABLED_DIMENSIONS]] | ||
| * if no custom value is specified. | ||
| * | ||
| * @param metricsEnabledDimensions [[Set[String]]] to specify | ||
| * the enabled CloudWatch metrics dimensions | ||
| * @return Reference to this [[KinesisInputDStream.Builder]] | ||
| * @see [[https://docs.aws.amazon.com/streams/latest/dev/monitoring-with-kcl.html#metric-levels]] | ||
|
||
| */ | ||
| // scalastyle:on line.size.limit | ||
| def metricsEnabledDimensions(metricsEnabledDimensions: Set[String]): Builder = { | ||
| this.metricsEnabledDimensions = Option(metricsEnabledDimensions) | ||
| this | ||
| } | ||
|
|
||
| /** | ||
| * Create a new instance of [[KinesisInputDStream]] with configured parameters and the provided | ||
| * message handler. | ||
|
|
@@ -287,7 +329,9 @@ object KinesisInputDStream { | |
| ssc.sc.clean(handler), | ||
| kinesisCredsProvider.getOrElse(DefaultCredentials), | ||
| dynamoDBCredsProvider, | ||
| cloudWatchCredsProvider) | ||
| cloudWatchCredsProvider, | ||
| metricsLevel.getOrElse(DEFAULT_METRICS_LEVEL), | ||
| metricsEnabledDimensions.getOrElse(DEFAULT_METRICS_ENABLED_DIMENSIONS)) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -324,4 +368,8 @@ object KinesisInputDStream { | |
| private[kinesis] val DEFAULT_KINESIS_REGION_NAME: String = "us-east-1" | ||
| private[kinesis] val DEFAULT_INITIAL_POSITION: KinesisInitialPosition = new Latest() | ||
| private[kinesis] val DEFAULT_STORAGE_LEVEL: StorageLevel = StorageLevel.MEMORY_AND_DISK_2 | ||
| private[kinesis] val DEFAULT_METRICS_LEVEL: MetricsLevel = | ||
| KinesisClientLibConfiguration.DEFAULT_METRICS_LEVEL | ||
| private[kinesis] val DEFAULT_METRICS_ENABLED_DIMENSIONS: Set[String] = | ||
| KinesisClientLibConfiguration.DEFAULT_METRICS_ENABLED_DIMENSIONS.asScala.toSet | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a small thing, but I know we have had problems generating scaladoc with references to library code. It might be worth building the doc HTML (only) as in https://github.com/apache/spark/blob/master/docs/README.md to make sure.
You don't really have to link a type like Set.
Also we usually do a continuation indent of two spaces.