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

Some local partitions's segments don't get purged when over retention time #596

Open
bingkunyangvungle opened this issue Sep 22, 2024 · 11 comments
Assignees

Comments

@bingkunyangvungle
Copy link

What happened?

We keep the local retention to be 20% of the total retention and the configuration looks like this:

      config = {
        "retention.ms"          = 86400000 / 8     #  3 hours 
        "local.retention.ms"    = 86400000 / 8 / 5 # 20% local data
        "remote.storage.enable" = true
      }

Normally for each partition there's about 9~11 segments stored locally, but sometimes for a certain partition, the cluster seems to 'forget' to delete the local segment that is out of the retention policy. As a result, the number of segments can grow continuously and the data size for the broker would go up non-stop as well, causing the issue of high disk utilization. After observing the issue, restart the Kafka service in the broker who is the leader for the partition with issue and the out-of-retention segments would be purged afterwards.

This is what happened before and after the restarting of the leader for the partition:
image

Kafka version: 3.7.0

Tiered Storage version: 2024-04-02-1712056402

What did you expect to happen?

The out-of-retention segments would be purged automatically.

What else do we need to know?

Not sure whether this is the issue for the Kafka or the plugin. So maybe this submit might be a good start for discussion.

@bingkunyangvungle
Copy link
Author

For the logs before the broker restart, there's only Rolled new log segment logs:

[2024-09-21 21:50:07,077] INFO [ProducerStateManager partition=topic-125] Wrote producer snapshot at offset 63538937203 with 0 producer ids in 1 ms. (org.apache.kafka.storage.internals.log.ProducerStateManager)
[2024-09-21 21:52:48,432] INFO [LocalLog partition=topic-125, dir=/data/kafka] Rolled new log segment at offset 63541597236 in 1 ms. (kafka.log.LocalLog)
[2024-09-21 21:52:48,432] INFO [ProducerStateManager partition=topic-125] Wrote producer snapshot at offset 63541597236 with 0 producer ids in 1 ms. (org.apache.kafka.storage.internals.log.ProducerStateManager)
[2024-09-21 21:55:29,915] INFO [LocalLog partition=topic-125, dir=/data/kafka] Rolled new log segment at offset 63544256614 in 0 ms. (kafka.log.LocalLog)

And this is the only logs found before restarting the broker for several hours, but no delete logs found.

@bingkunyangvungle bingkunyangvungle changed the title Some local partitions don't get purged when over retention time Some local partitions's segments don't get purged when over retention time Sep 22, 2024
@jeqo
Copy link
Contributor

jeqo commented Sep 24, 2024

Sounds related to https://issues.apache.org/jira/browse/KAFKA-16511 -- could you try this on 3.7.1 and see if it's still an issue?

@bingkunyangvungle
Copy link
Author

bingkunyangvungle commented Oct 10, 2024

Hi @jeqo , thank you for your response. Actually I tried 3.7.1 and the same issue happened again. This time I got more clear idea of how it happened. The local segment was successfully uploaded to S3, but it didn't get deleted locally. Also
this partition has way less segments than the other partitions in S3, because all the segments after it are "blocked" by this segment in this partition.
As for the logs, I only saw it was trying to upload, but didn't see the successful upload message for that segment:

Copying log segment data, metadata: RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=9SR61_KCT92N36mvlxeVfQ:topic-80

Then after I restart the broker, the "blocked" segment uploaded to S3 again and everything are back to normal(local disk was reduced as well).
image

As we are still trying to find the root cause, is there something you think that might cause this issue? Thank you.

@bingkunyangvungle
Copy link
Author

bingkunyangvungle commented Oct 10, 2024

One thing that worth mentioning is that: this not only happened in one broker, it actually happened in two brokers and both broker has the partition with issue. After I restarted the broker that holds the leader of the partition, the disk that held the follower of the partition also dropped.

@bingkunyangvungle
Copy link
Author

bingkunyangvungle commented Oct 13, 2024

Just give another datapoint:
The local segment that not gets deleted is uploaded to S3.
image

image
And the segment after it is also uploaded to S3.
Looks like is just that local segment not been cleaned up.

Then after restart the broker that has the leader of the segment, the deletion works:
image

So looks like it's a mixed situation here. Sometimes the segment is uploaded to S3 and sometimes it doesn't.

@bingkunyangvungle
Copy link
Author

bingkunyangvungle commented Oct 23, 2024

Hi @jeqo , It just happened again for 2 cases and looks like it just tried to upload, but either never succeeded to upload to S3 or it actually uploaded successfully, but it doesn't seem to "know" and no logs for the successful upload as well. These logs looks like:

[2024-10-23 02:50:00,549] INFO [RemoteLogManager=27 partition=FBmNXszXRo2dGWeQ9lNbKQ:topic-217] Copying 00000000013819732679.log to remote storage. (kafka.log.remote.RemoteLogManager$RLMTask)
[2024-10-23 02:50:00,552] INFO Copying log segment data, metadata: RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=FBmNXszXRo2dGWeQ9lNbKQ:topic-217, id=5mBo1GHnRDWyVUjYo1gFDw}, startOffset=13819732679, endOffset=13822372032, brokerId=27, maxTimestampMs=1729651777804, eventTimestampMs=1729651800549, segmentLeaderEpochs={16=13819732679}, segmentSizeInBytes=1073735369, customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED} (io.aiven.kafka.tieredstorage.RemoteStorageManager)

Instead what a successful upload would be:

[2024-10-23 02:47:05,812] INFO [RemoteLogManager=27 partition=FBmNXszXRo2dGWeQ9lNbKQ:user-217] Copying 00000000013817090668.log to remote storage. (kafka.log.remote.RemoteLogManager$RLMTask)
[2024-10-23 02:48:00,356] INFO [RemoteLogManager=27 partition=FBmNXszXRo2dGWeQ9lNbKQ:ex-jaeger-auction-report-20220421-217] Copied 00000000013817090668.log to remote storage with segment-id: RemoteLogSegmentId{topicIdPartition=FBmNXszXRo2dGWeQ9lNbKQ:user-217, id=GT7finQ_SgCRVu6idXHHgg} (kafka.log.remote.RemoteLogManager$RLMTask)

I was wondering if the plugin has the timeout for uploading to configure? For our case, it hanged there for about 2 hours already and if we have a timeout, maybe then we can re-upload it instead of kept waiting. Also if there's a mechanism to check instead of just waiting and doing nothing?

@jeqo
Copy link
Contributor

jeqo commented Oct 23, 2024

@bingkunyangvungle sorry for the late reply. I have failed to find some time to look into this. Will try again this week.

In the meantime, there are a couple of configs that you can try for timeouts on S3:

s3.api.call.timeout (including all retries) and s3.api.call.attempt.timeout (each retry). Let us know how it works with these settings. Also, could you share which commit for kafka and ts plugin are you using? Thanks!

@jeqo jeqo self-assigned this Oct 23, 2024
@bingkunyangvungle
Copy link
Author

bingkunyangvungle commented Oct 23, 2024

Hi @jeqo , thank you for your reply.
Just give more context of what is on my side:
I checked the corresponding code in Kafka, this line is getting executed based on my log, however there's no error or exception from this line, meaning that it has to go into this block and just stuck there. Since RemoteStorageManager is implemented by the plugin(this line appear in log), I guess we might want to take some time looking into this. If you can help on it, that'll be awesome!

BTW, the plugin version we are using is this one: 2024-04-02-1712056402
As for kafka, we tried 3.7.0 and 3.7.1 and both version has this issue.

Actually we also tried to set the size-based local retention, it works for other partition, but not for this one. It seems to me that it aware the segment has not uploaded, then it won't delete that one. (Just guessing)

@jeqo
Copy link
Contributor

jeqo commented Oct 23, 2024

@bingkunyangvungle
Copy link
Author

bingkunyangvungle commented Oct 23, 2024

Hi @jeqo , thank you for this one release. I noticed that the release still missing the artifacts (core and the s3/azure/google cloud), compared to the other releases. Do we have plan to upload them?
BTW, I noticed that there's a couple of changes that might fix for our situation, especially for:
1.Upload rate limit(code, config)
2.Removing orphan files(code)
So for the rate limit, do you have some suggestions for the threshold? Or is there something else relevant you would recommend to look at for this release? Thank you!

@jeqo
Copy link
Contributor

jeqo commented Oct 23, 2024

Argh, missed that. Uploading them now.
About the rate limit: I'd suggest to try with 100-200MiB/s and analyze the CPU usage.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

2 participants