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

KAFKA-15951: MissingSourceTopicException should include topic names #15573

Merged
merged 9 commits into from
May 10, 2024

Conversation

chickenchickenlove
Copy link
Contributor

@chickenchickenlove chickenchickenlove commented Mar 21, 2024

This is minor changes!

  • Jira : https://issues.apache.org/jira/browse/KAFKA-15951
  • MissingSourceTopicException has field missingTopics to store missing topics.
  • The StreamsPartitionAssignor throws a MissingSourceTopicException. and it depends on result of repartitionTopics.missingSourceTopicExceptions().isEmpty(). thus, missingSourceTopicExceptions() must always contain an iterable MissingSourceTopicException, allowing for the aggregation and throwing of MissingSourceTopics.
  • Idea for improving more
    • StreamsRebalanceListener can throw MissingSourceTopicException as well. however, StreamsRebalanceListener cannot get missing topic List because StreamsRebalanceListener depends on ErrorCode AssignorError.INCOMPLETE_SOURCE_TOPIC_METADATA.code()
    • If StreamsRebalanceListener should include missing topic list when it throws MissingSourceTopicException as well, we can consider ThreadLocal to reach the target.
    • Skeleton
      • Set ThreadLocal to StreamThread as member field.
      • Put missing topics to ThreadLocal.
      • StreamsRebalanceListener can get missing topics through ThreadLocal..

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Copy link
Contributor

@cadonna cadonna 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 the PR @chickenchickenlove !

Here my comments!

Comment on lines 29 to 38
this.missingTopics = new HashSet<>();
}

public MissingSourceTopicException(final String message, final Set<String> missingTopics) {
super(message);
this.missingTopics = missingTopics;
}

public Set<String> getMissingTopics() {
return this.missingTopics;
Copy link
Contributor

Choose a reason for hiding this comment

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

This class is part of the public API. That means, we cannot change it without a Kafka Improvement Proposal (KIP)[1].
I am not sure if adding the missing source topics to the exception makes too much sense, because the exception is caught in the StreamsPartitionAssignor [2] and transformed to a group assignment error (INCOMPLETE_SOURCE_TOPIC_METADATA). The missing source topics are not propagated to the point where the actual MissingSourceTopicException is thrown to the users, which is in the StreamsRebalanceListener [3]. To achieve this, a protocol change would be needed which I think it is not worth.
What we could do instead is log an error message with the missing source topics in RepartitionsTopics or StreamsPartitionAssignor. I slightly prefer the latter.

[1] https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
[2]


[3]
throw new MissingSourceTopicException("One or more source topics were missing during rebalance");

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hi, @cadonna. Thanks for your comment and your time. 🙇‍♂️
It is very helpful for me, and i feel quite sorry to make you spend a lot of time for me.

After reading the KIP document, now i can tell public interface should be introduced carefully!
I have a comment to make new commit to apply your comment.

When you have some free time, could you take a look? it will be very helpful for me. 🙇‍♂️

Copy link
Contributor

Choose a reason for hiding this comment

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

Don't worry! We are also here to guide new contributors.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@cadonna thanks for your comment, really.
your comment encourages me really 🙇‍♂️

Comment on lines 122 to 123
missingSourceTopics, subtopologyId, topologyName),
missingSourceTopics),
Copy link
Contributor

Choose a reason for hiding this comment

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

Would it be possible to return a set of missing source topics from RepartitionTopics?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@cadonna, I have a question. ✋
May i use the package-private access modifier for this?
I think that package-private access modifier seems not be included to public interface, right? (https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals)

If so, i think that solution will be quite simple.

Copy link
Contributor Author

@chickenchickenlove chickenchickenlove Mar 25, 2024

Choose a reason for hiding this comment

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

FYI, i write skeleton code below!
Does it make sense to you as well?

public class RepartitionTopics {
    ...
    // Add new field (private)
    private final Set<String> missingTopics = new HashSet(); 

    ...
    public Set<String> topologiesWithMissingInputTopics() { ... }
    public Queue<StreamsException> missingSourceTopicExceptions() { ... }

    // Add new method (package-private)
    Set<String> getMissingTopics() {
      return this.missingTopics;
    }
   ...
}

Copy link
Contributor

@cadonna cadonna Mar 26, 2024

Choose a reason for hiding this comment

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

The public interface is defined as everything that shows up in the javadocs. Classes in a package whose name contains internals do not show up in the javadocs. Class RepartitionTopics is in package org.apache.kafka.streams.processor.internals. Thus, RepartitionTopics is not part of the public interface.
Regarding field missingTopics, there is already missingInputTopicsBySubtopology that includes all missing topics.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I really appreciate your kind explanation. 👍
I've understood it clearly, and thanks to you, I've established the correct direction for revision.

@@ -523,7 +523,17 @@ private RepartitionTopics prepareRepartitionTopics(final Cluster metadata) {
final boolean isMissingInputTopics = !repartitionTopics.missingSourceTopicExceptions().isEmpty();
Copy link
Contributor

Choose a reason for hiding this comment

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

Here you could then use the method on RepartitionTopcs I proposed in my other comment.

@@ -523,7 +523,17 @@ private RepartitionTopics prepareRepartitionTopics(final Cluster metadata) {
final boolean isMissingInputTopics = !repartitionTopics.missingSourceTopicExceptions().isEmpty();
if (isMissingInputTopics) {
if (!taskManager.topologyMetadata().hasNamedTopologies()) {
throw new MissingSourceTopicException("Missing source topics.");
Copy link
Contributor

Choose a reason for hiding this comment

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

In addition to throwing the exception you would also log the error.

Copy link
Member

Choose a reason for hiding this comment

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

Was Bruno meant was, that we need to add log.error(...) to log the error message before throwing the exception. Seems you did not add this yet?

Copy link
Member

Choose a reason for hiding this comment

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

Adding to a previous comment from Bruno, I am wondering if we should also change the error log in StreamsRebalanceListener to point out that the missing source topic names might be logged on a different instance?

The StreamsRebalanceListener is executed on every instance, but StreamsPartitionAssignor only on the group leader.

Copy link
Contributor Author

@chickenchickenlove chickenchickenlove Mar 28, 2024

Choose a reason for hiding this comment

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

@mjsax thanks for your comments!

Was Bruno meant was, that we need to add log.error(...) to log the error message before throwing the exception. Seems you did not add this yet?

No, i created new commits. Please refer to images below 🙇‍♂️
In this Image, Bruno saidIn addition to throwing the exception you would also log the error.

image

and then, I have made these changes to reflect that comment to maintain readability.
you can see this commit.
image

Copy link
Contributor Author

@chickenchickenlove chickenchickenlove Mar 28, 2024

Choose a reason for hiding this comment

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

Adding to a previous comment from Bruno, I am wondering if we should also change the error log in StreamsRebalanceListener to point out that the missing source topic names might be logged on a different instance?

The StreamsRebalanceListener is executed on every instance, but StreamsPartitionAssignor only on the group leader.

@mjsax , IMHO, currently with this PR alone, it is not possible to refer to Missing Topicson StreamRebalanceListener.

To get Missing topics on StreamRebalanceListener as well, i wrote suggestion on this PR. Please refer to Idea for improving more on this PR description. I think it can be done without modifying public API.

Also, i will create some images for detail. Wait a sec, please.

Copy link
Contributor Author

@chickenchickenlove chickenchickenlove Mar 28, 2024

Choose a reason for hiding this comment

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

image I draw image above to describe `idea for improve`. If you want to read `MissingSourceTopics` in StreamRebalanceListener as well, this workaround seems like it could be a way to handle it.
  1. As you know, ThreadLocal provides storage specific to each thread.
  2. Both StreamThread, StreamRebalanceListener, StreamPartitionAssignor are included on internal package. that means, it is not public API.
  3. StreamRebalanceListener has reference of StreamThread Already.

this.rebalanceListener = new StreamsRebalanceListener(time, taskManager, this, this.log, this.assignmentErrorCode);

From this, I believe one StreamThread has own StreamRebalanceListener instance. thsu, ThreadLocal is suitable workaround in this case, i believe.

Thus, all the things that we should do, are 3step.

  1. Add ThreadLocal to field of StreamThread.
  2. Add method that put missing source topics to ThreadLocal before throw MissingSourceTopicExceptions,
  3. Add some codes on StreamRebalanceListener to get MissingSourceTopics from ThreadLocal.

Does it make sense to you?

Copy link
Member

Choose a reason for hiding this comment

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

Seems there is some misunderstanding. Sorry for causing confusion.

  1. This current code is not producing a log message:
final String errorMsg = String.format("Missing source topics. %s", repartitionTopics.missingSourceTopics());
throw new MissingSourceTopicException(errorMsg);

The code should be something like:

final String errorMsg = String.format("Missing source topics. %s", repartitionTopics.missingSourceTopics());
log.error(errorMsg);
throw new MissingSourceTopicException(errorMsg);
  1. I did not propose to include the topic names... As Bruno already pointed out, it would require a protocol change what seems to be overkill. In StreamsRebalanceListener, we currently log
Received error code 1

Cf https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java#L56

I would propose to actually change AssignorError to contain a proper String. Error code 1 does not mean anything to users. Additionally, it might be good to just change the error message of the log line and the exception to say something like: "To check which topics are missing, please look into the logs of the consumer group leader. Only the leaders knows and logs the name of the missing topics."

Copy link
Contributor Author

@chickenchickenlove chickenchickenlove Mar 28, 2024

Choose a reason for hiding this comment

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

@mjsax Thanks for your kindful descriptions. it was very helpful for me 🙇‍♂️

1.This current code is not producing a log message:

I misunderstood, thanks for your time!!
I added log.error() through this commit.

I would propose to actually change AssignorError to contain a proper String.

I agree with you.
Here's how I've implemented it based on your suggestion. (commit)
I don't know all the meaning of AssignorError codes exactly, so I referred to the existing error logs message related with AssignorError codes to write the messages.

What do you think? Do you think i'm on right direction?
When you have some free time, Please take a look 🙇‍♂️.

@chickenchickenlove
Copy link
Contributor Author

Gently ping, @cadonna !
I make new commit to apply your comments.
When you have some free time, please take a look. 🙇‍♂️

@mjsax mjsax added the streams label Mar 28, 2024
Copy link
Member

@mjsax mjsax 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 the PR. Made a pass.

@@ -523,7 +523,17 @@ private RepartitionTopics prepareRepartitionTopics(final Cluster metadata) {
final boolean isMissingInputTopics = !repartitionTopics.missingSourceTopicExceptions().isEmpty();
if (isMissingInputTopics) {
if (!taskManager.topologyMetadata().hasNamedTopologies()) {
throw new MissingSourceTopicException("Missing source topics.");
Copy link
Member

Choose a reason for hiding this comment

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

Was Bruno meant was, that we need to add log.error(...) to log the error message before throwing the exception. Seems you did not add this yet?

@@ -523,7 +523,17 @@ private RepartitionTopics prepareRepartitionTopics(final Cluster metadata) {
final boolean isMissingInputTopics = !repartitionTopics.missingSourceTopicExceptions().isEmpty();
if (isMissingInputTopics) {
if (!taskManager.topologyMetadata().hasNamedTopologies()) {
throw new MissingSourceTopicException("Missing source topics.");
Copy link
Member

Choose a reason for hiding this comment

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

Adding to a previous comment from Bruno, I am wondering if we should also change the error log in StreamsRebalanceListener to point out that the missing source topic names might be logged on a different instance?

The StreamsRebalanceListener is executed on every instance, but StreamsPartitionAssignor only on the group leader.

@chickenchickenlove
Copy link
Contributor Author

Thanks for the PR. Made a pass.

Aye, i hope so. 👍

@chickenchickenlove
Copy link
Contributor Author

Gently ping, @mjsax .
Would you please take a look when you have some free time? 🙇‍♂️

Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

@chickenchickenlove -- sorry for stalling to review this further.

Overall LGTM. A few suggested for the error code description.

chickenchickenlove and others added 4 commits May 2, 2024 19:56
…nals/assignment/AssignorError.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…nals/assignment/AssignorError.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…nals/assignment/AssignorError.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…nals/assignment/AssignorError.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
@chickenchickenlove
Copy link
Contributor Author

@mjsax thanks for your suggestion. it is very suitable 👍
I commit your suggestion. when you have free time, please take a look 🙇‍♂️

@mjsax
Copy link
Member

mjsax commented May 3, 2024

There is a checkstyle error:

[2024-05-02T11:03:05.489Z] > Task :streams:checkstyleMain

[2024-05-02T11:03:05.489Z] [ant:checkstyle] [ERROR] /home/jenkins/workspace/Kafka_kafka-pr_PR-15573/streams/src/main/java/org/apache/kafka/streams/processor/internals/RepartitionTopics.java:113:45: '{' is not preceded with whitespace. [WhitespaceAround]

@mjsax
Copy link
Member

mjsax commented May 3, 2024

I took the liberty to commit a fix to retrigger the build.

@chickenchickenlove
Copy link
Contributor Author

Hi @mjsax !
Sorry, I missed to check checkStyle and thanks for your working instead of me 🙇‍♂️

@mjsax mjsax merged commit ff6d01c into apache:trunk May 10, 2024
1 check failed
@mjsax
Copy link
Member

mjsax commented May 10, 2024

Thanks for the PR @chickenchickenlove! Merged to trunk.

@chickenchickenlove
Copy link
Contributor Author

Thanks for your guideline.
It was very helpful to me 🙇‍♂️🙇‍♂️🙇‍♂️

gongxuanzhang pushed a commit to gongxuanzhang/kafka that referenced this pull request Jun 12, 2024
…pache#15573)

MissingSourceTopicException should contain the name of the missing topic.
There is one corner case for which we don't have the topic name at hand, but we can log the topic
name somewhere else.

Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants