-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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-4062][Streaming]Add ReliableKafkaReceiver in Spark Streaming Kafka connector #2991
Conversation
Hi @tdas , would you mind taking a look at this? Thanks a lot. |
Test build #22425 has started for PR 2991 at commit
|
Test build #22425 has finished for PR 2991 at commit
|
Test FAILed. |
Test build #22438 has started for PR 2991 at commit
|
Test build #22438 has finished for PR 2991 at commit
|
Test PASSed. |
09d57c5
to
bb57d05
Compare
Test build #22446 has started for PR 2991 at commit
|
Test build #22446 has finished for PR 2991 at commit
|
Test PASSed. |
@jerryshao Thanks for this change! I am taking a look at this. But the code does not merge. So please update the code to the master branch. |
OK, will do :). |
bb57d05
to
48752b2
Compare
Test build #22595 has started for PR 2991 at commit
|
Test build #22595 has finished for PR 2991 at commit
|
Test PASSed. |
Hi @tdas , would you mind reviewing this code? Thanks a lot. |
waitToPush() | ||
currentBuffer += data | ||
listener.onStoreData(data, metadata) |
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.
What is the point of calling onStore data right after inserting into the buffer? If something needs to be done right after insertion then that can be done in the caller class.
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.
The reason of putting onStoreData here is that this is synchronized with the same mutex, if we put this into caller class, we should use another mutex, this will add some other synchronization logic.
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.
Actually adding synch logic in caller is more obvious, and introduces minimum extra logic in the BlockGenerator, keeping the BlockGenerator more generic. It is tricky to document stuff like "method 1 and method 2 of BlockGenerator are called in the same lock, but not other methods".
I have made some high level comments, but I am going to do another deeper roudn to understand the logic and correctness. In the mean time, since the WAL has been merged, could you test whether this patch is working as expected with the WAL turned on? |
Thanks a lot for your review, I will address the above comments, besides we are under test with StreamBench, the result will return to you later. |
private lazy val blockOffsetMap = | ||
new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] | ||
|
||
private lazy val blockGeneratorListener = new BlockGeneratorListener { |
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.
Good to define the named class for this generator listener.
Test build #23142 has started for PR 2991 at commit
|
@jerryshao Here is another round of changes from me.
Besides I also eliminated more duplicate code form the unit tests and merged two unit tests to reduce test run times. Please take a look. |
I did more refactoring for Refactoring 2, to create jerryshao#8 . This is what I finally recommend for merging. Please take a look. I have run the 3 kafka testsuites repeatedly for a long time, and they seemed consistently pass. |
OK, I will, thanks a lot, greatly appreciated. |
Refactor 3 = Refactor 2 + refactored KafkaStreamSuite further to elimite KafkaTestUtils, and made Java testsuite more robust
Test build #23345 has started for PR 2991 at commit
|
Lets see if this passes jenkins, I hadnt tried that yet |
Hi TD, this test is so flaky, it fails several times in my local test:
|
Test build #23345 has finished for PR 2991 at commit
|
Test PASSed. |
Jenkins, retest this please. |
Test build #23352 has started for PR 2991 at commit
|
Test build #23352 has finished for PR 2991 at commit
|
Test PASSed. |
Jenkins, retest this please. |
Test build #23362 has started for PR 2991 at commit
|
Test build #23362 timed out for PR 2991 at commit |
Test FAILed. |
Jenkins, retest this please. |
Test build #23380 has started for PR 2991 at commit
|
Test build #23380 has finished for PR 2991 at commit
|
Test PASSed. |
Alright, I am merging this! Thanks @jerryshao for all the hard work. Though, I encourage you to continue testing this receiver on your side prior to the spark 1.2 release for stability and correctness. |
…afka connector Add ReliableKafkaReceiver in Kafka connector to prevent data loss if WAL in Spark Streaming is enabled. Details and design doc can be seen in [SPARK-4062](https://issues.apache.org/jira/browse/SPARK-4062). Author: jerryshao <[email protected]> Author: Tathagata Das <[email protected]> Author: Saisai Shao <[email protected]> Closes #2991 from jerryshao/kafka-refactor and squashes the following commits: 5461f1c [Saisai Shao] Merge pull request #8 from tdas/kafka-refactor3 eae4ad6 [Tathagata Das] Refectored KafkaStreamSuiteBased to eliminate KafkaTestUtils and made Java more robust. fab14c7 [Tathagata Das] minor update. 149948b [Tathagata Das] Fixed mistake 14630aa [Tathagata Das] Minor updates. d9a452c [Tathagata Das] Minor updates. ec2e95e [Tathagata Das] Removed the receiver's locks and essentially reverted to Saisai's original design. 2a20a01 [jerryshao] Address some comments 9f636b3 [Saisai Shao] Merge pull request #5 from tdas/kafka-refactor b2b2f84 [Tathagata Das] Refactored Kafka receiver logic and Kafka testsuites e501b3c [jerryshao] Add Mima excludes b798535 [jerryshao] Fix the missed issue e5e21c1 [jerryshao] Change to while loop ea873e4 [jerryshao] Further address the comments 98f3d07 [jerryshao] Fix comment style 4854ee9 [jerryshao] Address all the comments 96c7a1d [jerryshao] Update the ReliableKafkaReceiver unit test 8135d31 [jerryshao] Fix flaky test a949741 [jerryshao] Address the comments 16bfe78 [jerryshao] Change the ordering of imports 0894aef [jerryshao] Add some comments 77c3e50 [jerryshao] Code refactor and add some unit tests dd9aeeb [jerryshao] Initial commit for reliable Kafka receiver (cherry picked from commit 5930f64) Signed-off-by: Tathagata Das <[email protected]>
Yes, I will, thanks a lot, greatly appreciate your help. |
Add ReliableKafkaReceiver in Kafka connector to prevent data loss if WAL in Spark Streaming is enabled. Details and design doc can be seen in SPARK-4062.