mirror of https://github.com/apache/kafka.git
MINOR: fix flaky RemoteLogManagerTest#testStopPartitionsWithDeletion (#18474)
The test has become flakier recently and it's easy to reproduce by running the single test (vs running the the class test suite). The root cause is that following functions call `RemoteLogMetadataManager#listRemoteLogSegments`. It returns iterator. If one of function goes through iterator first, another can't get expected result. I changed `thenReturn` to `thenAnswer` to avoid the issue. The race is between: * RLMExpirationTask#cleanupExpiredRemoteLogSegments * RemoteLogManager#deleteRemoteLogPartition Reviewers: Ismael Juma <ismael@juma.me.uk> Signed-off-by: PoAn Yang <payang@apache.org>
This commit is contained in:
parent
b9a952df6c
commit
a7342a2e62
|
@ -2150,9 +2150,9 @@ public class RemoteLogManagerTest {
|
|||
partitions.add(new StopPartition(followerTopicIdPartition.topicPartition(), true, true, true));
|
||||
|
||||
when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition)))
|
||||
.thenReturn(listRemoteLogSegmentMetadata(leaderTopicIdPartition, 5, 100, 1024, RemoteLogSegmentState.DELETE_SEGMENT_FINISHED).iterator());
|
||||
.thenAnswer(invocation -> listRemoteLogSegmentMetadata(leaderTopicIdPartition, 5, 100, 1024, RemoteLogSegmentState.DELETE_SEGMENT_FINISHED).iterator());
|
||||
when(remoteLogMetadataManager.listRemoteLogSegments(eq(followerTopicIdPartition)))
|
||||
.thenReturn(listRemoteLogSegmentMetadata(followerTopicIdPartition, 3, 100, 1024, RemoteLogSegmentState.DELETE_SEGMENT_FINISHED).iterator());
|
||||
.thenAnswer(invocation -> listRemoteLogSegmentMetadata(followerTopicIdPartition, 3, 100, 1024, RemoteLogSegmentState.DELETE_SEGMENT_FINISHED).iterator());
|
||||
CompletableFuture<Void> dummyFuture = new CompletableFuture<>();
|
||||
dummyFuture.complete(null);
|
||||
when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any()))
|
||||
|
|
Loading…
Reference in New Issue