Skip to content

HOTFIX: fix failed case RemoteLogManagerTest#testStopPartitionsWithDeletion#18474

Merged
ijuma merged 1 commit intoapache:trunkfrom
FrankYang0529:hotfix-testStopPartitionsWithDeletion
Jan 10, 2025
Merged

HOTFIX: fix failed case RemoteLogManagerTest#testStopPartitionsWithDeletion#18474
ijuma merged 1 commit intoapache:trunkfrom
FrankYang0529:hotfix-testStopPartitionsWithDeletion

Conversation

@FrankYang0529
Copy link
Copy Markdown
Member

@FrankYang0529 FrankYang0529 commented Jan 10, 2025

Not sure which commit makes the case fail. The CI result of my PR fails with RemoteLogManagerTest#testStopPartitionsWithDeletion. The trunk branch can also reproduce this.

The root cause is that following functions call RemoteLogMetadataManager#listRemoteLogSegments. It returns iterator. If one of function go through iterator first, another can't get expected result. I change thenReturn to thenAnswer to avoid the issue.

  • RLMExpirationTask#cleanupExpiredRemoteLogSegments
  • RemoteLogManager#deleteRemoteLogPartition
> ./gradlew clean :core:test --tests "kafka.log.remote.RemoteLogManagerTest.testStopPartitionsWithDeletion"
> Task :core:test

Gradle Test Run :core:test > Gradle Test Executor 7 > RemoteLogManagerTest > testStopPartitionsWithDeletion() PASSED

Committer Checklist (excluded from commit message)

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

@github-actions github-actions Bot added core Kafka Broker tests Test fixes (including flaky tests) tiered-storage Related to the Tiered Storage feature small Small PRs labels Jan 10, 2025
…letion

Signed-off-by: PoAn Yang <payang@apache.org>
@FrankYang0529 FrankYang0529 force-pushed the hotfix-testStopPartitionsWithDeletion branch from 34ae90a to 12d9936 Compare January 10, 2025 07:58
@TaiJuWu
Copy link
Copy Markdown
Collaborator

TaiJuWu commented Jan 10, 2025

#18321 make this CI fail.
@ijuma Could you take a look?

Copy link
Copy Markdown
Collaborator

@TaiJuWu TaiJuWu left a comment

Choose a reason for hiding this comment

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

This can reproduce on my local. Thanks for the fix!

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 10, 2025

The PR you referenced was green when I merged it:

Screenshot_20250110-003435

@TaiJuWu
Copy link
Copy Markdown
Collaborator

TaiJuWu commented Jan 10, 2025

The PR you referenced was green when I merged it:

Yes, you're correct, but I used the following script to identify the commit that caused this test to fail, and it points to this specific commit if I am wrong please correct me.

git bisect start HEAD
git bisect bad
git bisect good 3f9d2c2db061a10fa9db20a88e4b509e34191eb5
git bisect run ./gradlew :core:test --tests "kafka.log.remote.RemoteLogManagerTest.testStopPartitionsWithDeletion"

But the weird thing is trunk is green after this PR got merged ...

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 10, 2025

Ok, that's weird if it's failing consistently. Also, I have another PR I was working on 30 minutes ago and it didn't show this failure either.

I'll take a closer look when I'm near a computer but that won't be for a few hours.

@TaiJuWu
Copy link
Copy Markdown
Collaborator

TaiJuWu commented Jan 10, 2025

Ok, that's weird if it's failing consistently. Also, I have another PR I was working on 30 minutes ago and it didn't show this failure either.

I'll take a closer look when I'm near a computer but that won't be for a few hours.

I total agree this is weird thing but still hope you can help to check since you are the author 🙇
Thanks for your help!

@FrankYang0529
Copy link
Copy Markdown
Member Author

The failed case AbstractCoordinatorTest#testWakeupAfterSyncGroupReceivedExternalCompletion is not related to this PR and already filed an issue https://issues.apache.org/jira/browse/KAFKA-18310.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 10, 2025

@TaiJuWu I don't think your approach to identifying the problematic commit is valid because this test is clearly flaky and not failing every time. You can see the following trunk run where it passed:

https://github.com/apache/kafka/actions/runs/12708697404/job/35427267406

I ran it locally multiple times and it also passed.

So, why do we think this is a recent problem versus a test that has been flaky?

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 10, 2025

Ah, interesting, I can reproduce this if I run only testStopPartitionsWithDeletion - when I run the class test suite, it always passed (for me) locally.

Copy link
Copy Markdown
Member

@ijuma ijuma 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 fix. It makes sense that we should return a new iterator each invocation instead of always the same iterator. It looks like this test bug has existed for a while.

A few things that are unclear:

  1. Why does it fail consistently when invoked in isolation (locally) but not when the class test suite is executed (locally or remotely)?
  2. Why did it start failing more recently - the relevant commit didn't change this test in a meaningful way.

I'll go ahead and merge in the meantime, however.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 10, 2025

It must be related to the race condition mentioned in the PR description, the case where it fails must be due to the race condition winner changing - i.e. this was always possible, but somehow is more likely under certain scenarios.

RLMExpirationTask#cleanupExpiredRemoteLogSegments
RemoteLogManager#deleteRemoteLogPartition

@ijuma ijuma merged commit a7342a2 into apache:trunk Jan 10, 2025
ijuma pushed a commit that referenced this pull request Jan 10, 2025
…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>
@FrankYang0529 FrankYang0529 deleted the hotfix-testStopPartitionsWithDeletion branch January 10, 2025 14:46
@TaiJuWu
Copy link
Copy Markdown
Collaborator

TaiJuWu commented Jan 10, 2025

@ijuma Thanks for your explanation. I also think this is flaky at the moment.
No matter what thanks for you notice this issue at the busy time!

ijuma added a commit to ijuma/kafka that referenced this pull request Jan 10, 2025
…emove-metadata-version-methods-for-versions-older-than-3.0

* apache-github/trunk:
  KAFKA-18340: Change Dockerfile to use log4j2 yaml instead log4j properties (apache#18378)
  MINOR: fix flaky RemoteLogManagerTest#testStopPartitionsWithDeletion (apache#18474)
  KAFKA-18311: Enforcing copartitioned topics (4/N) (apache#18397)
  KAFKA-18308; Update CoordinatorSerde (apache#18455)
  KAFKA-18440: Convert AuthorizationException to fatal error in AdminClient (apache#18435)
  KAFKA-17671: Create better documentation for transactions (apache#17454)
  KAFKA-18304; Introduce json converter generator (apache#18458)
  MINOR: Clean up classic group tests (apache#18473)
  KAFKA-18399 Remove ZooKeeper from KafkaApis (2/N): CONTROLLED_SHUTDOWN and ENVELOPE (apache#18422)
  MINOR: improve StreamThread periodic processing log (apache#18430)
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 10, 2025

Thanks for bringing it to my attention.

pranavt84 pushed a commit to pranavt84/kafka that referenced this pull request Jan 27, 2025
…pache#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>
manoj-mathivanan pushed a commit to manoj-mathivanan/kafka that referenced this pull request Feb 19, 2025
…pache#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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Kafka Broker small Small PRs tests Test fixes (including flaky tests) tiered-storage Related to the Tiered Storage feature

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants