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-22968][DStream] Throw an exception on partition revoking issue #21038

Closed
wants to merge 2 commits into from

Conversation

jerryshao
Copy link
Contributor

@jerryshao jerryshao commented Apr 11, 2018

What changes were proposed in this pull request?

Kafka partitions can be revoked when new consumers joined in the consumer group to rebalance the partitions. But current Spark Kafka connector code makes sure there's no partition revoking scenarios, so trying to get latest offset from revoked partitions will throw exceptions as JIRA mentioned.

Partition revoking happens when new consumer joined the consumer group, which means different streaming apps are trying to use same group id. This is fundamentally not correct, different apps should use different consumer group. So instead of throwing an confused exception from Kafka, improve the exception message by identifying revoked partition and directly throw an meaningful exception when partition is revoked.

Besides, this PR also fixes bugs in DirectKafkaWordCount, this example simply cannot be worked without the fix.

8/01/05 09:48:27 INFO internals.ConsumerCoordinator: Revoking previously assigned partitions [kssh-7, kssh-4, kssh-3, kssh-6, kssh-5, kssh-0, kssh-2, kssh-1] for group use_a_separate_group_id_for_each_stream
18/01/05 09:48:27 INFO internals.AbstractCoordinator: (Re-)joining group use_a_separate_group_id_for_each_stream
18/01/05 09:48:27 INFO internals.AbstractCoordinator: Successfully joined group use_a_separate_group_id_for_each_stream with generation 4
18/01/05 09:48:27 INFO internals.ConsumerCoordinator: Setting newly assigned partitions [kssh-7, kssh-4, kssh-6, kssh-5] for group use_a_separate_group_id_for_each_stream

How was this patch tested?

This is manually verified in local cluster, unfortunately I'm not sure how to simulate it in UT, so propose the PR without UT added.

@jerryshao jerryshao changed the title [SPARK-22968][DStream] Fix Kafka connector partition revoked issue [SPARK-22968][DStream] Fix Kafka partition revoked issue Apr 11, 2018
@SparkQA
Copy link

SparkQA commented Apr 11, 2018

Test build #89177 has finished for PR 21038 at commit f317dec.

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

@Hackeruncle
Copy link

@jerryshao Thank you very much for this issue.
I go to compile and test.

@jerryshao
Copy link
Contributor Author

@koeninger would you please help to review, thanks!

@koeninger
Copy link
Contributor

The log in the jira looks like it's from a consumer rebalance, i.e. more than one driver consumer was running with the same group id.

Isn't the underlying problem here that the user is creating multiple streams with the same group id, despite what the documentation says? The log even says s/he copy-pasted the documentation group id "group use_a_separate_group_id_for_each_stream"

I don't think we should silently "fix" that. As a user, I wouldn't expect app A to suddenly start processing only half of the partitions just because entirely different app B started with the (misconfigured) same group id.

@jerryshao
Copy link
Contributor Author

Thanks @koeninger for your comments. I think your suggestion is valid, the log here is just pasted from JIRA, but we also got the same issue from customer's report.

Here in the PR description, I mentioned that using two apps with same group id to mimic this issue. But I'm not sure the real use case from our customer, maybe in their scenario such usage is valid.

So I'm wondering if we can add a configuration to control whether it should be fail or just warning. Also I think exception/warning log should be improved to directly tell user about consumer rebalance issue, rather than throwing from Kafka as "no current assignment for partition xxx".

@koeninger
Copy link
Contributor

koeninger commented Apr 12, 2018 via email

@jerryshao
Copy link
Contributor Author

Thanks @koeninger , then I will just improve the exception message.

@jerryshao jerryshao changed the title [SPARK-22968][DStream] Fix Kafka partition revoked issue [SPARK-22968][DStream] Throw an exception on partition revoking issue Apr 12, 2018
@SparkQA
Copy link

SparkQA commented Apr 12, 2018

Test build #89230 has finished for PR 21038 at commit a7770e9.

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

@jerryshao
Copy link
Contributor Author

Ping @koeninger , would you please help to review again. Thanks!

@koeninger
Copy link
Contributor

Seems like that should help address the confusion. Merging to master.

@asfgit asfgit closed this in 5fccdae Apr 18, 2018
@jerryshao
Copy link
Contributor Author

Thanks @koeninger for the review.

@SehanRathnayake
Copy link

I think "This is fundamentally not correct, different apps should use different consumer group" statement is wrong. .According to Kafka Having consumers as part of the same consumer group means providing the “competing consumers” pattern with whom the messages from topic partitions are spread across the members of the group.
It seams Spark-Fafka cannot support this for now

@koeninger
Copy link
Contributor

@SehanRathnayake Kafka is designed for at most one consumer per partition per consumer group at any given point in time, https://kafka.apache.org/documentation/#design_consumerposition
Spark already manages creating a consumer per partition for the consumer group associated with a stream.
If you have a valid use case for running multiple Spark applications with the same consumer group, please explain it in a jira, not discussion of a pull request that has already been merged.

@anandchangediya
Copy link

anandchangediya commented Oct 14, 2019

@koeninger According to Kafka documentation

If all the consumer instances have the same consumer group, then the records will effectively be load-balanced over the consumer instances
This means I can have multiple consumers with the same groupId which can help me to load balance my application and scale accordingly.
I don't know why it is said "fundamentally wrong" to have multiple consumers with the same groupId in spark.
So how can I achieve scalability to listen to a single partition and increase consumption rate with multiple spark consumers?
Is this the spark design fault or any other way to achieve scaling which I am unaware of?

@SehanRathnayake Any thoughts?

@koeninger
Copy link
Contributor

koeninger commented Oct 14, 2019 via email

@anandchangediya
Copy link

Hey @koeninger thanks for the reply
The issue with my application is I have one topic with 3 partitions once I start my application (Spark consumer) it listens to all the 3 partitions
LOG found as below
Setting newly assigned partitions [topic.partition-2, topic.partition-1, topic.partition-0]

When I start another instance of the same application with the same group id I can see there is rebalance in spark and one partition is assigned to the second application instance

LOG as follows in the first application instance

Setting newly assigned partitions [topic.partition-2, topic.partition-0]

So we can see the topic.partition 1 is assigned to the second instance of an application in the rebalancing process
But just after the above-mentioned log there is an exception as follows

java.lang.IllegalStateException: Previously tracked partitions [topic.partition-1] been revoked by Kafka because of consumer rebalance. This is mostly due to another stream with same group id joined, please check if there're different streaming application misconfigure to use the same group id. Fundamentally different stream should use different group id

And the application exits.
How can I have multiple consumers with same groupId for different partitions?
I also provided Assingmnt strategy as RoundRobin
kafkaParam.put("partition.assignment.strategy", "org.apache.kafka.clients.consumer.RoundRobinAssignor");

@koeninger
Copy link
Contributor

koeninger commented Oct 14, 2019 via email

@joanjiao2016
Copy link

@koeninger Hi, we have prepared two spark streaming applications with the same group id to run respectively on different cluster for disaster recovery,the first application will failed when the second application started a few minutes later, and threw exception as:
java.lang.IllegalStateException: No current assignment for partition xxx

@koeninger
Copy link
Contributor

Why can't you use a different group id?

@joanjiao2016
Copy link

Why can't you use a different group id?

If the two spark streaming applications use different group id, the data will be processed twice and the result (in hbase) will be wrong

@koeninger
Copy link
Contributor

You already have to handle data being processed twice, or you're getting bad results in the event of a failure.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
7 participants