Skip to content

[SPARK-20140][DStream] Remove hardcoded kinesis retry wait and max retries#17467

Closed
yssharma wants to merge 1 commit intoapache:masterfrom
yssharma:ysharma/spark-kinesis-retries
Closed

[SPARK-20140][DStream] Remove hardcoded kinesis retry wait and max retries#17467
yssharma wants to merge 1 commit intoapache:masterfrom
yssharma:ysharma/spark-kinesis-retries

Conversation

@yssharma
Copy link
Copy Markdown

What changes were proposed in this pull request?

The pull requests proposes to remove the hardcoded values for Amazon Kinesis - MIN_RETRY_WAIT_TIME_MS, MAX_RETRIES.

This change is critical for kinesis checkpoint recovery when the kinesis backed rdd is huge.
Following happens in a typical kinesis recovery :

  • kinesis throttles large number of requests while recovering
  • retries in case of throttling are not able to recover due to the small wait period
  • kinesis throttles per second, the wait period should be configurable for recovery

The patch picks the spark kinesis configs from:

  • spark.streaming.kinesis.retry.wait.time
  • spark.streaming.kinesis.retry.max.attempts

Jira : https://issues.apache.org/jira/browse/SPARK-20140

How was this patch tested?

Modified the KinesisBackedBlockRDDSuite.scala to run kinesis tests with the modified configurations. Wasn't able to test the patch with actual throttling.

@HyukjinKwon
Copy link
Copy Markdown
Member

Hi @yssharma, this PR looks a non-trivial change that needs a JIRA. Please refer http://spark.apache.org/contributing.html.

@yssharma
Copy link
Copy Markdown
Author

Have got this JIRA ticket for the patch - https://issues.apache.org/jira/browse/SPARK-20140

@yssharma
Copy link
Copy Markdown
Author

Oh right you meant the PR title format. I will reject this pr and post a new one. Thanks @HyukjinKwon

@HyukjinKwon
Copy link
Copy Markdown
Member

You could just edit the title. I think closing this and opening new one is also fine and an option.

@yssharma yssharma changed the title Ysharma/spark kinesis retries [SPARK-20140][DStream] Mar 29, 2017
@yssharma yssharma changed the title [SPARK-20140][DStream] [SPARK-20140][DStream] Remove hardcoded kinesis retry wait and max retries Mar 29, 2017
@yssharma
Copy link
Copy Markdown
Author

@HyukjinKwon What should be the next steps for this PR. Are there any Spark-Kinesis experts who can review the patch ?

@HyukjinKwon
Copy link
Copy Markdown
Member

I am not used to Kinesis. I usually click blame button and check both the recent code modifier and committer, e.g., https://github.com/yssharma/spark/blame/4b589adeaef540f6227266ecc628ad41ef0733c3/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala

I think I assume @tdas and @brkyvz are experts in this area.

@yssharma
Copy link
Copy Markdown
Author

yssharma commented Mar 30, 2017

You're a gem @HyukjinKwon 💯 . I will wait for Tathagata and Burak's inputs then :)

yssharma referenced this pull request Mar 31, 2017
…sient annotations

The Scala 2.11 SBT build currently fails for Spark 1.6.0 and master due to warnings about the `transient` annotation:

```
[error] [warn] /Users/joshrosen/Documents/spark/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala:73: no valid targets for annotation on value sc - it is discarded unused. You may specify targets with meta-annotations, e.g. (transient param)
[error] [warn]     transient sc: SparkContext,
```

This fix implemented here is the same as what we did in #8433: remove the `transient` annotations when they are not necessary and replace use  `transient private val` in the remaining cases.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10479 from JoshRosen/fix-sbt-2.11.
@yssharma yssharma force-pushed the ysharma/spark-kinesis-retries branch from 4b589ad to 67bcf26 Compare March 31, 2017 03:49
@yssharma
Copy link
Copy Markdown
Author

@tdas @brkyvz

  • Added new changes for removing other constants hardcoded at multiple places.
  • Squashed 3 commits into single commit

@yssharma yssharma force-pushed the ysharma/spark-kinesis-retries branch 4 times, most recently from 89e27fd to ccb6c19 Compare April 3, 2017 07:12
@yssharma
Copy link
Copy Markdown
Author

yssharma commented Apr 3, 2017

Can I get some feedback here please @tdas @brkyvz Thanks :)

@SparkQA
Copy link
Copy Markdown

SparkQA commented Apr 3, 2017

Test build #3634 has finished for PR 17467 at commit ccb6c19.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yssharma
Copy link
Copy Markdown
Author

yssharma commented Apr 4, 2017

@srowen - Could I get some love here as well. Thanks

Copy link
Copy Markdown
Member

@srowen srowen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems reasonable but I don't know enough to review Kinesis changes, really

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

define this once during class initialization

Copy link
Copy Markdown
Contributor

@brkyvz brkyvz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for working on this PR! Sorry for not being able to take a look earlier. Some feedback:

  1. I thought the kinesisConfigs map was there to make testing easier, but it didn't since you set the SparkConfs as well as the map, so I suggest we remove it
  2. Let's make spark.streaming.kinesis.retry.wait.time a duration string, allow it to take 100ms instead of have it be in millis. You can use https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java#L244
  3. May be a tad bit shorter to name as:
    spark.streaming.kinesis.retry.waitTime
    spark.streaming.kinesis.retry.maxAttempts

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

define this once during class initialization as well

Comment thread docs/streaming-kinesis-integration.md Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can take duration strings, such as 100ms or 1s, I would prefer we use that.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do you need this to be provided as a Map? You already have sc being passed in with all the configurations

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1. I think reading the config values from sc will be a much cleaner approach.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@yssharma yssharma force-pushed the ysharma/spark-kinesis-retries branch from ccb6c19 to f51367f Compare April 8, 2017 11:55
@yssharma
Copy link
Copy Markdown
Author

yssharma commented Apr 8, 2017

@brkyvz - thanks for taking time to review the patch. appreciate it.
Implemented all your suggestions. Now passing a new map for the kinesis configs and added mechanism to use the builder for the configs.

As for the spark context, I wanted to use the sparkcontext available in KinesisBackedBlockRDD directly as well (instead of creating a new config map), but the sc in KinesisBackedBlockRDD
is not available, and trying to use it there causes serialization errors. Passing a different config map looked like the only simple solution to access the kineses configs.

The patch now doesnot use the sc at all and expects a kinesisConf to be passed to the KinesisInputDStream builder directly.

Let me know your thoughts. Thanks again for the review comments.

@yssharma
Copy link
Copy Markdown
Author

yssharma commented Apr 8, 2017

Just for info, while trying to use the sc in the KinesisBackedBlockRDD in the older patch:

- Basic reading from Kinesis *** FAILED *** org.apache.spark.SparkException: Task not serializable at org.apache.spark.util.ClosureCleaner$.ensureSerializable(ClosureCleaner.scala:298) at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:288) at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:108) at org.apache.spark.SparkContext.clean(SparkContext.scala:2284) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2058) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2084) at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:936) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) at org.apache.spark.rdd.RDD.withScope(RDD.scala:362) ... Cause: java.io.NotSerializableException: org.apache.spark.SparkContext Serialization stack: - object not serializable (class: org.apache.spark.SparkContext, value: org.apache.spark.SparkContext@60c1663c) - field (class: org.apache.spark.streaming.kinesis.KinesisBackedBlockRDD, name: org$apache$spark$streaming$kinesis$KinesisBackedBlockRDD$$sc, type: class org.apache.spark.SparkContext) - object (class org.apache.spark.streaming.kinesis.KinesisBackedBlockRDD, KinesisBackedBlockRDD[0] at BlockRDD at KinesisBackedBlockRDD.scala:90) - field (class: org.apache.spark.NarrowDependency, name: _rdd, type: class org.apache.spark.rdd.RDD)

@yssharma
Copy link
Copy Markdown
Author

Waiting for review @brkyvz . Thanks.

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't this be a val?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, this value is modified after waits -
kinesisWaitTimeMs *= 2 // if you have waited, then double wait time for next round

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think you want to do this-- kinesisWaitTimeMs is never reset to the default value after the retry loop exists. I think you should make this a val and introduce a var initialized to its value within retryOrTimeout() to store the wait time for each retry iteration.

@budde
Copy link
Copy Markdown

budde commented Apr 20, 2017

Not a Spark committer, but I've contributed to this component in the past. I would strongly prefer an approach that avoids adding an additional parameter to all of the Kinesis classes if the SparkConf from sc can be used instead. I haven't looked at the initial version of your code, but based on the stacktrace you've posted it seems like you might've been referencing sc directly from code running running on the executor (e.g. the KinesisSequenceRangeIterator instance created in the compute() method via getBlockFromKinesis()). Did you try simply extracting the two config values from sc and storing them as fields for KinesisBackedBlockRDDPartition then passing them as constructor arguments to KinesisSequenceRangeIterator?

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Documentation is not correct

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If using SparkConf to store these custom config values doesn't end up being feasible then I'd strongly prefer that we follow the existing approach and have separate builder methods for setting the retry wait time and max attempts.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you think it would be better to pass values to builder rather than a map of configs. I thought map of configs can be easily extended when we need to support new configurations without code changes?
What is your thought on values+builder per config vs one map for all configs ?

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you want the extensibility of a key/value map for configs then I would go the route of getting a solution that uses SparkConf to do that in order to use the existing facilities provided by Spark. It doesn't make sense to me to introduce a key/value map just for Kinesis, especially since the naming of your keys (e.g. spark.streaming.kinesis.retry.waitTime) would indicate to most users that these are SparkConf params, not a Kinesis-specific mapping that must be manually set up and passed to the Kinesis stream builder.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s"$message

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also please move + to line above

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about something like this?

private[kinesis] case class KinesisReadConfigurations(
    maxRetries: Int,
    retryWaitTimeMs: Long,
    retryTimeoutMs: Long)

object KinesisReadConfigurations {
  def apply(): KinesisReadConfigurations = {
    KinesisReadConfigurations(3, 100, 10000)
  }

  def apply(ssc: StreamingContext): KinesisReadConfigurations = {
    KinesisReadConfigurations(
      maxRetries = ssc.sc.getConf.getInt(RETRY_MAX_ATTEMPTS_KEY, DEFAULT_MAX_RETRIES),
      retryWaitTimeMs = JavaUtils.timeStringAsMs(
        ssc.sc.getConf.getString(RETRY_WAIT_TIME_KEY, DEFAULT_RETRY_WAIT_TIME)),
      retryTimeoutMs = ssc.graph.batchDuration.milliseconds)
  }

  val DEFAULT_MAX_RETRIES = 3
  val DEFAULT_RETRY_WAIT_TIME = "100ms"
}

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

mind putting these in a try - finally block

@brkyvz
Copy link
Copy Markdown
Contributor

brkyvz commented May 3, 2017

@yssharma left some comments

@yssharma yssharma force-pushed the ysharma/spark-kinesis-retries branch 2 times, most recently from 6a61218 to 6a2950f Compare May 4, 2017 13:02
@yssharma
Copy link
Copy Markdown
Author

yssharma commented May 4, 2017

@brkyvz - Thanks for the review comments. Updated the patch, please review.

Copy link
Copy Markdown

@budde budde left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Took a look and left a few comments, mostly around some style and formatting issues. On the whole, I'm not sure how I feel about introducing the KinesisReadConfigurations case class if it's just being used to pull a couple config values out from SparkConf and pass them to KinesisBackedBlockRDD which itself is already taking a SparkContext as an argument. Reading the config values in the constructor and storing them as val fields should resolve the serialization issues you were seeing while avoiding having to serialize the entire SparkConf. I'll defer to @brkyvz on this issue though.

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: You're missing a space here and on the following line

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Incorrect indentation here-- should be 2 softabs/4 spaces

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Copy Markdown

@budde budde May 4, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would use constants and named parameters here too, e.g.

def apply(): KinesisReadConfigurations = KinesisReadConfigurations(
  maxRetries = DEFAULT_MAX_RETRIES,
  ...

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: should be [[KinesisBackedBlockRDD]]

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Copy Markdown
Contributor

@brkyvz brkyvz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking much better now. Left a couple more comments

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

actually do we even need this?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It can be used in places where we don't have the spark conf. I am using this in KinesisBackedBlockRDD's constructor.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

private object

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

these also need to be in the finally

@yssharma
Copy link
Copy Markdown
Author

yssharma commented May 4, 2017

Thanks for the comments @budde @brkyvz . Would be adding the changes soon.
I too liked pulling the values of spark conf directly and got it working with the private val in KinesisBackedBlockRDD [1]. I don't mind getting the conf from the case class either since it keeps all the configs in a place and the class acts as a self documented code. Open to thoughts from you both.

  1. yssharma@f5026b4

@brkyvz
Copy link
Copy Markdown
Contributor

brkyvz commented May 5, 2017

I really prefer the case class. It's used in places where we don't pass in SparkContext as well

@budde
Copy link
Copy Markdown

budde commented May 5, 2017

Fair enough. I took another look and I think I may have been thinking of the way things worked in an earlier revision of this code. I think the case class is reasonable.

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it would be sufficient to change this to

  kinesisReadConfigs = KinesisReadConfigurations(ssc))

and omit lines 65-70. I don't think a comment is necessary here, the code is pretty straightforward.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@yssharma yssharma force-pushed the ysharma/spark-kinesis-retries branch from 6a2950f to 91a777c Compare May 6, 2017 04:03
@yssharma
Copy link
Copy Markdown
Author

yssharma commented May 6, 2017

Thanks for all the review comments @budde @brkyvz . Added new review changes.

@yssharma
Copy link
Copy Markdown
Author

@budde @brkyvz - Any feed back on this one please ?

Copy link
Copy Markdown

@budde budde left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, apologies for the delay

@brkyvz
Copy link
Copy Markdown
Contributor

brkyvz commented May 16, 2017

LGTM! Merging to master/branch-2.2

asfgit pushed a commit that referenced this pull request May 16, 2017
…tries

## What changes were proposed in this pull request?

The pull requests proposes to remove the hardcoded values for Amazon Kinesis - MIN_RETRY_WAIT_TIME_MS, MAX_RETRIES.

This change is critical for kinesis checkpoint recovery when the kinesis backed rdd is huge.
Following happens in a typical kinesis recovery :
- kinesis throttles large number of requests while recovering
- retries in case of throttling are not able to recover due to the small wait period
- kinesis throttles per second, the wait period should be configurable for recovery

The patch picks the spark kinesis configs from:
- spark.streaming.kinesis.retry.wait.time
- spark.streaming.kinesis.retry.max.attempts

Jira : https://issues.apache.org/jira/browse/SPARK-20140

## How was this patch tested?

Modified the KinesisBackedBlockRDDSuite.scala to run kinesis tests with the modified configurations. Wasn't able to test the patch with actual throttling.

Author: Yash Sharma <ysharma@atlassian.com>

Closes #17467 from yssharma/ysharma/spark-kinesis-retries.

(cherry picked from commit 38f4e86)
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
@asfgit asfgit closed this in 38f4e86 May 16, 2017
@yssharma
Copy link
Copy Markdown
Author

Awesome. Thanks @budde @brkyvz for reviews and patch improvements.

ghost pushed a commit to dbtsai/spark that referenced this pull request May 18, 2017
## What changes were proposed in this pull request?

The changes were merged as part of - apache#17467.
The documentation was missed somewhere in the review iterations. Adding the documentation where it belongs.

## How was this patch tested?
Docs. Not tested.

cc budde , brkyvz

Author: Yash Sharma <ysharma@atlassian.com>

Closes apache#18028 from yssharma/ysharma/kinesis_retry_docs.
asfgit pushed a commit that referenced this pull request May 18, 2017
## What changes were proposed in this pull request?

The changes were merged as part of - #17467.
The documentation was missed somewhere in the review iterations. Adding the documentation where it belongs.

## How was this patch tested?
Docs. Not tested.

cc budde , brkyvz

Author: Yash Sharma <ysharma@atlassian.com>

Closes #18028 from yssharma/ysharma/kinesis_retry_docs.

(cherry picked from commit 92580bd)
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
robert3005 pushed a commit to palantir/spark that referenced this pull request May 19, 2017
…tries

## What changes were proposed in this pull request?

The pull requests proposes to remove the hardcoded values for Amazon Kinesis - MIN_RETRY_WAIT_TIME_MS, MAX_RETRIES.

This change is critical for kinesis checkpoint recovery when the kinesis backed rdd is huge.
Following happens in a typical kinesis recovery :
- kinesis throttles large number of requests while recovering
- retries in case of throttling are not able to recover due to the small wait period
- kinesis throttles per second, the wait period should be configurable for recovery

The patch picks the spark kinesis configs from:
- spark.streaming.kinesis.retry.wait.time
- spark.streaming.kinesis.retry.max.attempts

Jira : https://issues.apache.org/jira/browse/SPARK-20140

## How was this patch tested?

Modified the KinesisBackedBlockRDDSuite.scala to run kinesis tests with the modified configurations. Wasn't able to test the patch with actual throttling.

Author: Yash Sharma <ysharma@atlassian.com>

Closes apache#17467 from yssharma/ysharma/spark-kinesis-retries.
lycplus pushed a commit to lycplus/spark that referenced this pull request May 24, 2017
…tries

## What changes were proposed in this pull request?

The pull requests proposes to remove the hardcoded values for Amazon Kinesis - MIN_RETRY_WAIT_TIME_MS, MAX_RETRIES.

This change is critical for kinesis checkpoint recovery when the kinesis backed rdd is huge.
Following happens in a typical kinesis recovery :
- kinesis throttles large number of requests while recovering
- retries in case of throttling are not able to recover due to the small wait period
- kinesis throttles per second, the wait period should be configurable for recovery

The patch picks the spark kinesis configs from:
- spark.streaming.kinesis.retry.wait.time
- spark.streaming.kinesis.retry.max.attempts

Jira : https://issues.apache.org/jira/browse/SPARK-20140

## How was this patch tested?

Modified the KinesisBackedBlockRDDSuite.scala to run kinesis tests with the modified configurations. Wasn't able to test the patch with actual throttling.

Author: Yash Sharma <ysharma@atlassian.com>

Closes apache#17467 from yssharma/ysharma/spark-kinesis-retries.
lycplus pushed a commit to lycplus/spark that referenced this pull request May 24, 2017
## What changes were proposed in this pull request?

The changes were merged as part of - apache#17467.
The documentation was missed somewhere in the review iterations. Adding the documentation where it belongs.

## How was this patch tested?
Docs. Not tested.

cc budde , brkyvz

Author: Yash Sharma <ysharma@atlassian.com>

Closes apache#18028 from yssharma/ysharma/kinesis_retry_docs.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants