Skip to content
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

[SPARK-18020][Streaming][Kinesis] Checkpoint SHARD_END to finish reading closed shards #16213

Closed
wants to merge 7 commits into from

Conversation

maropu
Copy link
Member

@maropu maropu commented Dec 8, 2016

What changes were proposed in this pull request?

This pr is to fix an issue occurred when resharding Kinesis streams; the resharding makes the KCL throw an exception because Spark does not checkpoint SHARD_END when finishing reading closed shards in KinesisRecordProcessor#shutdown. This bug finally leads to stopping subscribing new split (or merged) shards.

How was this patch tested?

Added a test in KinesisStreamSuite to check if it works well when splitting/merging shards.

@SparkQA
Copy link

SparkQA commented Dec 8, 2016

Test build #69859 has finished for PR 16213 at commit 7c51da8.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@maropu
Copy link
Member Author

maropu commented Dec 8, 2016

I'm looking into the failure.

@SparkQA
Copy link

SparkQA commented Dec 8, 2016

Test build #69860 has finished for PR 16213 at commit a5739b3.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 8, 2016

Test build #69863 has finished for PR 16213 at commit e8a4e1d.

  • This patch fails from timeout after a configured wait of `250m`.
  • This patch merges cleanly.
  • This patch adds no public classes.

@maropu
Copy link
Member Author

maropu commented Dec 9, 2016

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Dec 9, 2016

Test build #69902 has finished for PR 16213 at commit e8a4e1d.

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

*/
public class CheckpointerShim {

public static void shutdown(IRecordProcessorCheckpointer checkpointer)
Copy link
Contributor

Choose a reason for hiding this comment

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

Have you tried this call? https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java#L129

It seems you may add a finalize method to KinesisCheckpointer where you call the IRecordProcessorCheckpointer.checkpoint(ExtendedSequenceNumber.SHARD_END.sequenceNumber, ExtendedSequenceNumber.SHARD_END.subSequenceNumber)

Copy link
Member Author

@maropu maropu Dec 10, 2016

Choose a reason for hiding this comment

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

Yea, I tried by this version, but I got the same exception;

16/12/10 05:49:00 ERROR ShutdownTask: Application exception.
java.lang.IllegalArgumentException: Sequence number must be numeric, but was SHARD_END
        at com.amazonaws.services.kinesis.clientlibrary.lib.worker.SequenceNumberValidator.validateSequenceNumber(SequenceNumberValidator.java:75)
        at com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer.checkpoint(RecordProcessorCheckpointer.java:120)
        at org.apache.spark.streaming.kinesis.KinesisCheckpointer.removeCheckpointer(KinesisCheckpointer.scala:77)
        at org.apache.spark.streaming.kinesis.KinesisReceiver.removeCheckpointer(KinesisReceiver.scala:258)

IIUC this is a kind of aws library bugs. I asked in the kinesis forum here, but I didn't get an answer.

"\nData received does not match data sent after splitting a shard")
}

val (shardToMerge, adjShared) = splitOpenShards match { case Seq(e1, e2) => (e1, e2) }
Copy link
Contributor

Choose a reason for hiding this comment

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

cleaner to do
val Seq(shardToMerge, adjShared) = splitOpenShards

Copy link
Contributor

@brkyvz brkyvz Dec 9, 2016

Choose a reason for hiding this comment

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

probably you also meant adjShard. I would use adjacentShard. For a moment I was thinking adjective, shared adjective, what does that mean :P

Copy link
Member Author

Choose a reason for hiding this comment

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

Yea, I tried to use adjacentShard, but it returned null. I didn't why it behaved like this though, do you know that?

@brkyvz
Copy link
Contributor

brkyvz commented Dec 9, 2016

Left a couple comments. I don't like the idea of using KCL internal methods. Hopefully we won't need to.

@maropu
Copy link
Member Author

maropu commented Dec 10, 2016

Thanks for these comments! ya, I do not like this approach, too. But, since those who reshard streams always hit this issue and resharding is important for load-balancing in Kinesis streams (recently, a new API UpdteShardCount has been implemented for resharding), I'd like to fix this as soon as possible, but I couldn't find better approaches than this.

@SparkQA
Copy link

SparkQA commented Dec 10, 2016

Test build #69956 has finished for PR 16213 at commit adf4dd6.

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

@SparkQA
Copy link

SparkQA commented Jan 12, 2017

Test build #71260 has finished for PR 16213 at commit 1b89a0a.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 12, 2017

Test build #71263 has finished for PR 16213 at commit 7f8336c.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 12, 2017

Test build #71268 has finished for PR 16213 at commit 5f0e632.

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

@maropu
Copy link
Member Author

maropu commented Jan 12, 2017

@brkyvz I asked AWS guys and I found correct handling in this case (See: https://forums.aws.amazon.com/thread.jspa?threadID=244218). I fixed in this way and could you check this again? Thanks!

@maropu
Copy link
Member Author

maropu commented Jan 15, 2017

@brkyvz ping

@brkyvz
Copy link
Contributor

brkyvz commented Jan 18, 2017

Sorry, taking a look

@maropu
Copy link
Member Author

maropu commented Jan 18, 2017

Many thanks!

}
ssc.start()

val (testData1, testData2, testData3) = (1 to 10, 11 to 20, 21 to 30)
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: instead of doing a tuple expansion, mind making these new lines

Copy link
Member Author

Choose a reason for hiding this comment

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

okay

"\nData received does not match data sent after splitting a shard")
}

val Seq(shardToMerge, adjShared) = splitOpenShards
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: adjShared -> adjShard?

@brkyvz
Copy link
Contributor

brkyvz commented Jan 18, 2017

This LGTM now! Thank you so much for checking with AWS and posting thread link as a comment.
cc @tdas to take a final look and possibly merge

@SparkQA
Copy link

SparkQA commented Jan 18, 2017

Test build #71555 has finished for PR 16213 at commit b7ef9a9.

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

@maropu
Copy link
Member Author

maropu commented Jan 19, 2017

@tdas ping

1 similar comment
@maropu
Copy link
Member Author

maropu commented Jan 21, 2017

@tdas ping

@maropu
Copy link
Member Author

maropu commented Jan 24, 2017

@brkyvz @tdas ping

@Gauravshah
Copy link
Contributor

facing same issue randomly on prod, can I help in some way to push it ?

// We must call `checkpoint()` with no parameter to finish reading shards.
// See an URL below for details:
// https://forums.aws.amazon.com/thread.jspa?threadID=244218
KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100)
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you run this if checkpointer != null. The docs mention that the checkpointer may be null.

Copy link
Member Author

Choose a reason for hiding this comment

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

Does the latest fix satisfy your intention?

@brkyvz
Copy link
Contributor

brkyvz commented Jan 25, 2017

@maropu One last comment. Then I'll merge this!

@maropu
Copy link
Member Author

maropu commented Jan 26, 2017

okay, I'll update soon!

@brkyvz
Copy link
Contributor

brkyvz commented Jan 26, 2017

LGTM! Pending tests.

@maropu
Copy link
Member Author

maropu commented Jan 26, 2017

Many thanks! Also, congrats, commiter

@SparkQA
Copy link

SparkQA commented Jan 26, 2017

Test build #72007 has finished for PR 16213 at commit 17a5c3a.

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

@brkyvz
Copy link
Contributor

brkyvz commented Jan 26, 2017

Thanks a lot! Merging to master! (May take a while, going to be my first merge!)

@maropu
Copy link
Member Author

maropu commented Jan 26, 2017

great, haha! Many thanks!

@asfgit asfgit closed this in 256a3a8 Jan 26, 2017
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…ing closed shards

## What changes were proposed in this pull request?
This pr is to fix an issue occurred when resharding Kinesis streams; the resharding makes the KCL throw an exception because Spark does not checkpoint `SHARD_END` when finishing reading closed shards in `KinesisRecordProcessor#shutdown`. This bug finally leads to stopping subscribing new split (or merged) shards.

## How was this patch tested?
Added a test in `KinesisStreamSuite` to check if it works well when splitting/merging shards.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes apache#16213 from maropu/SPARK-18020.
cmonkey pushed a commit to cmonkey/spark that referenced this pull request Feb 15, 2017
…ing closed shards

## What changes were proposed in this pull request?
This pr is to fix an issue occurred when resharding Kinesis streams; the resharding makes the KCL throw an exception because Spark does not checkpoint `SHARD_END` when finishing reading closed shards in `KinesisRecordProcessor#shutdown`. This bug finally leads to stopping subscribing new split (or merged) shards.

## How was this patch tested?
Added a test in `KinesisStreamSuite` to check if it works well when splitting/merging shards.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes apache#16213 from maropu/SPARK-18020.
tnixon pushed a commit to tnixon/spark that referenced this pull request Apr 14, 2017
…ing closed shards

## What changes were proposed in this pull request?
This pr is to fix an issue occurred when resharding Kinesis streams; the resharding makes the KCL throw an exception because Spark does not checkpoint `SHARD_END` when finishing reading closed shards in `KinesisRecordProcessor#shutdown`. This bug finally leads to stopping subscribing new split (or merged) shards.

## How was this patch tested?
Added a test in `KinesisStreamSuite` to check if it works well when splitting/merging shards.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes apache#16213 from maropu/SPARK-18020.
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.

4 participants