KAFKA-16712: Fix race in TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest#15962
KAFKA-16712: Fix race in TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest#15962chia7712 merged 3 commits intoapache:trunkfrom
Conversation
…bscriptionsTest `TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest` has a race when it sets the `RemoteLogMetadataTopicPartitioner` using the setter. This change fixes the race condition by passing the `RemoteLogMetadataTopicPartitioner` instance in a `Function<Integer, RemoteLogMetaedataTopicPartitioner>` which is used in `configure()` in `TopicBasedRemoteLogMetadataManager`. It also improves the waitingFor condition by spying on `RemotePartitionMetadataStore` and awaiting on Phasers to ensure ConsumerManager makes progress before performing assertions.
|
CC: @kamalcph this is following the discussion at #15885 (comment) |
kamalcph
left a comment
There was a problem hiding this comment.
LGTM. Thanks for the patch!
This is the first time, Phaser is being used in the project. And, it is not straight-forward to understand. Could you please add more comments?
|
|
||
| topicBasedRemoteLogMetadataManager.configure(configs); | ||
| if (remoteLogMetadataTopicPartitioner != null) { | ||
| topicBasedRemoteLogMetadataManager.setRlmTopicPartitioner(remoteLogMetadataTopicPartitioner); |
There was a problem hiding this comment.
setRlmTopicPartitioner gets unused now, so could you please remove it?
| Phaser initializationPhaser = new Phaser(2); // 1 to register test thread, 1 to register leaderTopicIdPartition | ||
| doAnswer(invocationOnMock -> { | ||
| Object result = invocationOnMock.callRealMethod(); | ||
| initializationPhaser.arriveAndDeregister(); |
There was a problem hiding this comment.
Maybe we can use CountDownLatch instead? And we can override the stub (i.e create doAnswer again) before running onPartitionLeadershipChanges? For example:
CountDownLatch latch = new CountDownLatch(1);
doAnswer(invocationOnMock -> {
Object result = invocationOnMock.callRealMethod();
latch.countDown();
return result;
}).when(spyRemotePartitionMetadataStore).markInitialized(any());
...
latch.await(30_000, TimeUnit.MILLISECONDS);
...
CountDownLatch latch2 = new CountDownLatch(1);
doAnswer(invocationOnMock -> {
Object result = invocationOnMock.callRealMethod();
latch2.countDown();
return result;
}).when(spyRemotePartitionMetadataStore).markInitialized(any());
...
rlmm().onPartitionLeadershipChanges(Collections.singleton(emptyTopicIdPartition),
Collections.singleton(followerTopicIdPartition));There was a problem hiding this comment.
I've added some comments in 9cfda4f to describe the Phaser operations following Kamal's suggestion.
Please let me know if that's okay. I'd prefer not duplicating the stub given a data structure in the standard library fits the purpose but I'm happy to change it if you feel it's confusing for future readers.
chia7712
left a comment
There was a problem hiding this comment.
+1 will merge it if no related failed tests
…scriptionsTest (apache#15962) TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest has a race when it sets RemoteLogMetadataTopicPartitioner using the setter. This change fixes the race condition by passing the RemoteLogMetadataTopicPartitioner instance in a Function<Integer, RemoteLogMetaedataTopicPartitioner> which is used in configure() in TopicBasedRemoteLogMetadataManager. It also improves the waitingFor condition by spying on RemotePartitionMetadataStore and awaiting on Phasers to ensure ConsumerManager makes progress before performing assertions. Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
…scriptionsTest (apache#15962) TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest has a race when it sets RemoteLogMetadataTopicPartitioner using the setter. This change fixes the race condition by passing the RemoteLogMetadataTopicPartitioner instance in a Function<Integer, RemoteLogMetaedataTopicPartitioner> which is used in configure() in TopicBasedRemoteLogMetadataManager. It also improves the waitingFor condition by spying on RemotePartitionMetadataStore and awaiting on Phasers to ensure ConsumerManager makes progress before performing assertions. Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
…scriptionsTest (apache#15962) TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest has a race when it sets RemoteLogMetadataTopicPartitioner using the setter. This change fixes the race condition by passing the RemoteLogMetadataTopicPartitioner instance in a Function<Integer, RemoteLogMetaedataTopicPartitioner> which is used in configure() in TopicBasedRemoteLogMetadataManager. It also improves the waitingFor condition by spying on RemotePartitionMetadataStore and awaiting on Phasers to ensure ConsumerManager makes progress before performing assertions. Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTesthas a race when it setsRemoteLogMetadataTopicPartitionerusing the setter.This change fixes the race condition by passing the
RemoteLogMetadataTopicPartitionerinstance in aFunction<Integer, RemoteLogMetaedataTopicPartitioner>which is used inconfigure()inTopicBasedRemoteLogMetadataManager.It also improves the waitingFor condition by spying on
RemotePartitionMetadataStoreand awaiting on Phasers to ensure ConsumerManager makes progress before performing assertions.Committer Checklist (excluded from commit message)