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-9769: Finish operations for leaderEpoch-updated partitions up to point ZK Exception #8479

Merged
merged 12 commits into from Jul 6, 2020

Conversation

andrewchoi5
Copy link
Contributor

@andrewchoi5 andrewchoi5 commented Apr 14, 2020

KAFKA-9769: Finish operations for leaderEpoch-updated partitions up to point ZK Exception occurs.
https://issues.apache.org/jira/browse/KAFKA-9769

For example, in such case, we will have the following mechanism :
1 - P1 and P2 succeeds. leaderEpoch for them are incremented because no ZkException occurs
2 - while making follower for P3, ZkException occurs and the leaderEpoch is not updated and thus thepartitionsToMakeFollower += partition isn’t executed. We catch this ZkException in line 1498 and log it as an error. No Exception is thrown.
3 - After catching the exception, makeFollower for P4 is then not executed.
4 - so the partitionsToMakeFollower only contains P1, P2. And fetchers are added to these partitionsToMakeFollower

Signed-off-by: Andrew Choi li_andchoi@microsoft.com

Committer Checklist (excluded from commit message)

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

…xception occurs

1 - Remove fetchers for partitions whose leader epoch is updated.
2 - Finish delayed fetch and produce requests for those same partitions
3 - Re-add fetchers for those same partitions.
4 - Don't throw exception, but rather log it as an error about this occurrence.

Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
@andrewchoi5
Copy link
Contributor Author

@mjsax Hello Matthias! I would appreciate your review. Thanks in advance!

@mjsax
Copy link
Member

mjsax commented Apr 19, 2020

I am not really familiar with this part of the code. Maybe @hachikuji or @cmccabe can help?

@andrewchoi5
Copy link
Contributor Author

Thanks for referring Matthias. Would appreciate your review @hachikuji @cmccabe

@andrewchoi5
Copy link
Contributor Author

@mjsax Hi Matthias, would you happen to know if there were any other reviewers available? I don't mind waiting, but was curious what the ETA usually appears to be.

@mjsax
Copy link
Member

mjsax commented May 8, 2020

Feel to reach out to the dev mailing list and call for review.

@hachikuji
Copy link
Contributor

@andrewchoi5 Just want to make sure I understand the problem. The scenario is that we lose the zk session while handling a LeaderAndIsr request. Current LeaderAndIsr handling works like this:

  1. Check epoch of each partition. If it is less than or equal to current epoch, ignore the update.
  2. Update the epoch for each partition.
  3. Make followers and leaders, which involves loading topic configs from zk.

So the problem occurs when we hit an error loading the topic configs in step 3). This potentially causes us to miss the needed state changes from that request and also prevents us from being able to retry them because the epoch has already been updated. Is that right?

I guess the fix here is only a partial fix. We would still be left with the one failed partition, right?

Off the top of my head, I am wondering whether we should be retrying the request at a lower level. For example, maybe getEntityConfigs could catch the ZooKeeperClientExpiredException and retry. I assume there is a good reason we do not catch this exception in retryRequestUntilConnected already. Perhaps it is unsafe to assume that the requests are still valid after a session expiration. However, just for reading configurations, I do not see a problem retrying after a session expiration.

cc @junrao

@junrao
Copy link
Contributor

junrao commented May 13, 2020

@andrewchoi5 : Thanks for finding this issue. We fixed https://issues.apache.org/jira/browse/KAFKA-9932 recently. So, the fetchLogConfig from ZK method is called rarely during makeFollowers() now. If we do hit the ZK exception, there are a few options: (1) As Jason mentioned, we could keep retrying from ZK until successful. This probably needs to be done in a separate thread to avoid blocking the request handler thread. So, it can be a bit involved. (2) Don't update the leader epoch (we can choose to update the leader epoch after the local log is obtained successfully) and log an error. Since this issue should be rare now, maybe we can do (2) for now?

@andrewchoi5
Copy link
Contributor Author

Hi @junrao. Thank you for your review -- I have further synced up with @jjkoshy on this PR.

The partition's new leader epoch update is actually happening after the point at which ZooKeeper Exception is thrown. Therefore, when the createLogs throws ZooKeeper Exception, the new leader epoch does not actually get updated.

In the catch case for ZooKeeperClientException, I have populated the responseMap with the topic partition and the Errors.NETWORK_EXCEPTION. If you suggest any other Error to be populated in this responseMap, please let me know and I will change it accordingly.

Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
@jjkoshy
Copy link
Contributor

jjkoshy commented Jun 12, 2020

In the catch case for ZooKeeperClientException, I have populated the responseMap with the topic partition and the Errors.NETWORK_EXCEPTION. If you suggest any other Error to be populated in this responseMap, please let me know and I will change it accordingly.

I haven't looked at this code in a while so I may not have enough context at this point, but I don't think we should use the network exception error code - i.e., this isn't a network issue between the coordinator and broker but between the broker and zk. Also, there doesn't seem to be any active retry attempt from the controller to resend the request in this scenario.

@andrewchoi5
Copy link
Contributor Author

In the catch case for ZooKeeperClientException, I have populated the responseMap with the topic partition and the Errors.NETWORK_EXCEPTION. If you suggest any other Error to be populated in this responseMap, please let me know and I will change it accordingly.

I haven't looked at this code in a while so I may not have enough context at this point, but I don't think we should use the network exception error code - i.e., this isn't a network issue between the coordinator and broker but between the broker and zk. Also, there doesn't seem to be any active retry attempt from the controller to resend the request in this scenario.

Correct -- I wasn't able to find the best, close enough Errors exception to populate, especially since there was none related to ZooKeeper in that class.

@junrao
Copy link
Contributor

junrao commented Jun 16, 2020

@andrewchoi5 : Since the controller only checks KAFKA_STORAGE_ERROR in LeaderAndIsrResponse now, perhaps we can just log an error without sending an error code back for now.

Andrew Choi added 3 commits June 15, 2020 18:43
…KAFKA_STORAGE_ERROR in LeaderAndIsrResponse now, just log an error without sending an error code back for now.

Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
@andrewchoi5
Copy link
Contributor Author

Thanks, @junrao .
Removed the population of responseMap with the ERROR code.

@andrewchoi5
Copy link
Contributor Author

Hello @junrao @hachikuji -- I have made some updates to address the comments. Thanks!

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@andrewchoi5 : Thanks for the updated PR. One more comment.

core/src/main/scala/kafka/server/ReplicaManager.scala Outdated Show resolved Hide resolved
Andrew Choi added 2 commits June 25, 2020 17:04
Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@andrewchoi5 : Thanks for the updated PR. A few more comments below.

core/src/main/scala/kafka/cluster/Partition.scala Outdated Show resolved Hide resolved
core/src/main/scala/kafka/cluster/Partition.scala Outdated Show resolved Hide resolved
Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
@andrewchoi5 andrewchoi5 requested a review from junrao June 26, 2020 03:50
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@andrewchoi5 : Thanks for the updated PR. One more comment below.

core/src/main/scala/kafka/cluster/Partition.scala Outdated Show resolved Hide resolved
Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
@andrewchoi5
Copy link
Contributor Author

Thanks for the review @junrao -- let me know if there's anything else for revision.

@andrewchoi5
Copy link
Contributor Author

Closed by accident.

@andrewchoi5 andrewchoi5 reopened this Jun 29, 2020
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@andrewchoi5 : Thanks for the new update. Still one more comment below.

core/src/main/scala/kafka/cluster/Partition.scala Outdated Show resolved Hide resolved
Copy link
Contributor Author

@andrewchoi5 andrewchoi5 left a comment

Choose a reason for hiding this comment

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

@junrao -- thanks for the comment. left a reply.

Signed-off-by: Andrew Choi <li_andchoi@microsoft.com>
@andrewchoi5
Copy link
Contributor Author

@junrao -- let me know if anything else needs your attention. Thanks!

@andrewchoi5 andrewchoi5 requested a review from junrao July 4, 2020 05:58
@junrao
Copy link
Contributor

junrao commented Jul 5, 2020

ok to test

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@andrewchoi5 : Thanks for the latest PR. LGTM

@junrao junrao merged commit 3a55fb9 into apache:trunk Jul 6, 2020
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
5 participants