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-15479: Remote log segments should be considered once for retention breach #14407
Conversation
@clolov @divijvaidya @showuon @satishd Call for review. PTAL. |
@@ -993,7 +988,9 @@ private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, Ex | |||
return; | |||
} | |||
RemoteLogSegmentMetadata metadata = segmentsIterator.next(); | |||
|
|||
if (segmentsToDelete.contains(metadata)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is the main fix.
I had a comment to fix this/similar problem in original PR - #13561 (comment) I am curious, why didn't the test which was added to resolve the comment fail? Is it because the test only checked for eligibility of a segment for calculation of size and didn't actually check if same segment is being counted twice? |
We don't have a test for the scenario mentioned in the comment. And, there was a regression after #14349. Added a couple of tests to verify the deleted segment count and log-start-offset. |
I will review this in a couple of hours, apologies for the delay |
The test failures are unrelated to this PR. |
…ion breach When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments lesser than expected. This is a follow-up of KAFKA-15352
6c06624
to
307d677
Compare
@clolov @divijvaidya @showuon @satishd Test failures are unrelated. Please take another look! |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good to me! Test failures are unrelated.
[Build / JDK 11 and Scala 2.13 / kafka.api.ConsumerBounceTest.testSeekAndCommitWithBrokerFailures()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14407/6/testReport/junit/kafka.api/ConsumerBounceTest/Build___JDK_11_and_Scala_2_13___testSeekAndCommitWithBrokerFailures__/)
[Build / JDK 11 and Scala 2.13 / org.apache.kafka.streams.integration.NamedTopologyIntegrationTest.shouldRemoveOneNamedTopologyWhileAnotherContinuesProcessing()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14407/6/testReport/junit/org.apache.kafka.streams.integration/NamedTopologyIntegrationTest/Build___JDK_11_and_Scala_2_13___shouldRemoveOneNamedTopologyWhileAnotherContinuesProcessing__/)
[Build / JDK 20 and Scala 2.13 / org.apache.kafka.common.network.SslTransportLayerTest.[2] tlsProtocol=TLSv1.2, useInlinePem=true](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14407/6/testReport/junit/org.apache.kafka.common.network/SslTransportLayerTest/Build___JDK_20_and_Scala_2_13____2__tlsProtocol_TLSv1_2__useInlinePem_true/)
[Build / JDK 20 and Scala 2.13 / kafka.api.TransactionsTest.testBumpTransactionalEpoch(String).quorum=kraft](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14407/6/testReport/junit/kafka.api/TransactionsTest/Build___JDK_20_and_Scala_2_13___testBumpTransactionalEpoch_String__quorum_kraft/)
[Build / JDK 20 and Scala 2.13 / org.apache.kafka.streams.integration.NamedTopologyIntegrationTest.shouldBackOffTaskAndEmitDataWithinSameTopology()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14407/6/testReport/junit/org.apache.kafka.streams.integration/NamedTopologyIntegrationTest/Build___JDK_20_and_Scala_2_13___shouldBackOffTaskAndEmitDataWithinSameTopology__/)
[Build / JDK 17 and Scala 2.13 / kafka.server.DescribeClusterRequestTest.testDescribeClusterRequestExcludingClusterAuthorizedOperations(String).quorum=kraft](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14407/6/testReport/junit/kafka.server/DescribeClusterRequestTest/Build___JDK_17_and_Scala_2_13___testDescribeClusterRequestExcludingClusterAuthorizedOperations_String__quorum_kraft/)
[Build / JDK 17 and Scala 2.13 / org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14407/6/testReport/junit/org.apache.kafka.trogdor.coordinator/CoordinatorTest/Build___JDK_17_and_Scala_2_13___testTaskRequestWithOldStartMsGetsUpdated__/)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @kamalcph for the PR and addressing the comments. LGTM.
Merging it to trunk as the failed tests are not related to the changes introduced in this PR. |
…ion breach (apache#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
…ion breach (apache#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
…ion breach (apache#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
…ion breach (#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
…ion breach (apache#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
…ion breach (apache#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
…ion breach (apache#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
…ion breach (apache#14407) (#33) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org> Co-authored-by: Kamal Chandraprakash <kchandraprakash@uber.com>
…ion breach (apache#14407) When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352 Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments lesser than expected. This is a follow-up of KAFKA-15352
Committer Checklist (excluded from commit message)