[ 
https://issues.apache.org/jira/browse/KAFKA-16511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17843174#comment-17843174
 ] 

Luke Chen commented on KAFKA-16511:
-----------------------------------

[~ckamal] , I was reading the tickets and your PR, and I still think this issue 
is because the log start offset has incremented in the local 
leader-epoch-checkpoint, but the remote storage is stored the old one. What I 
can see is:

The log segment in the remote storage:
{code:java}
startOffset=2971163, endOffset=2978396, segmentLeaderEpochs={7=2971163}{code}

Current local leader-epoch-checkpoint
{code:java}
$ cat leader-epoch-checkpoint 
0
2
7 2976337
21 2978397{code}

So, the start offset for epoch 7 is now 2976337 ( > 2971163 in remote storage).

 

And we expected the segment can be deleted in `isSegmentBreachByLogStartOffset` 
 [here 
|https://github.com/apache/kafka/blob/240243b91d69c2b65b5e456065fdcce90c121b04/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L903],
 but it won't:

 
{code:java}
Integer firstEpoch = leaderEpochEntries.firstKey();
shouldDeleteSegment = 
metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch <= 
firstEpoch) && metadata.endOffset() < logStartOffset;{code}
 

The firstEpoch is 7, and the 
`metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch <= 
firstEpoch)` will be true, but `metadata.endOffset() < logStartOffset` will be 
false (metadata.endOffset(): 2978396 , logStartOffset: 2976337 )

 

That is, this segment is not deleted in isSegmentBreachByLogStartOffset, and 
the latter `isRemoteSegmentWithinLeaderEpochs` will also return false for this 
segment. So, this segment is kept in the remote storage.

 

I think this segment should not be deleted by isSegmentBreachByLogStartOffset 
because the logEndOffset of that segment is still greater than logStartOffset. 
So, we should be able to let this segment be deleted (or first accepted) in the 
following `isRemoteSegmentWithinLeaderEpochs`.

Does that make sense?

> Leaking tiered segments
> -----------------------
>
>                 Key: KAFKA-16511
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16511
>             Project: Kafka
>          Issue Type: Bug
>          Components: Tiered-Storage
>    Affects Versions: 3.7.0
>            Reporter: Francois Visconte
>            Assignee: Kamal Chandraprakash
>            Priority: Major
>              Labels: tiered-storage
>
> I have some topics there were not written since a few days (having 12h 
> retention) where some data remains on tiered storage (in our case S3) and 
> they are never deleted.
>  
> Looking at the log history, it appears that we never even tried to delete 
> these segments:
> When looking at one of the non-leaking segment, I get the following 
> interesting messages:
>  
> {code:java}
> "2024-04-02T10:30:45.265Z","""kafka""","""10039""","[RemoteLogManager=10039 
> partition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764] Deleted remote log segment 
> RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764, 
> id=fqGng3UURCG3-v4lETeLKQ} due to leader-epoch-cache truncation. Current 
> earliest-epoch-entry: EpochEntry(epoch=8, startOffset=2980106), 
> segment-end-offset: 2976819 and segment-epochs: [5]"
> "2024-04-02T10:30:45.242Z","""kafka""","""10039""","Deleting log segment data 
> for completed successfully 
> RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId
> {topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764, 
> id=fqGng3UURCG3-v4lETeLKQ}
> , startOffset=2968418, endOffset=2976819, brokerId=10029, 
> maxTimestampMs=1712009754536, eventTimestampMs=1712013411147, 
> segmentLeaderEpochs={5=2968418}, segmentSizeInBytes=536351075, 
> customMetadata=Optional.empty, state=COPY_SEGMENT_FINISHED}"
> "2024-04-02T10:30:45.144Z","""kafka""","""10039""","Deleting log segment data 
> for RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId
> {topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764, 
> id=fqGng3UURCG3-v4lETeLKQ}
> , startOffset=2968418, endOffset=2976819, brokerId=10029, 
> maxTimestampMs=1712009754536, eventTimestampMs=1712013411147, 
> segmentLeaderEpochs={5=2968418}, segmentSizeInBytes=536351075, 
> customMetadata=Optional.empty, state=COPY_SEGMENT_FINISHED}"
> "2024-04-01T23:16:51.157Z","""kafka""","""10029""","[RemoteLogManager=10029 
> partition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764] Copied 
> 00000000000002968418.log to remote storage with segment-id: 
> RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764, 
> id=fqGng3UURCG3-v4lETeLKQ}"
> "2024-04-01T23:16:51.147Z","""kafka""","""10029""","Copying log segment data 
> completed successfully, metadata: 
> RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId
> {topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764, 
> id=fqGng3UURCG3-v4lETeLKQ}
> , startOffset=2968418, endOffset=2976819, brokerId=10029, 
> maxTimestampMs=1712009754536, eventTimestampMs=1712013397319, 
> segmentLeaderEpochs={5=2968418}, segmentSizeInBytes=536351075, 
> customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}"
> "2024-04-01T23:16:37.328Z","""kafka""","""10029""","Copying log segment data, 
> metadata: RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId
> {topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764, 
> id=fqGng3UURCG3-v4lETeLKQ}
> , startOffset=2968418, endOffset=2976819, brokerId=10029, 
> maxTimestampMs=1712009754536, eventTimestampMs=1712013397319, 
> segmentLeaderEpochs={5=2968418}, segmentSizeInBytes=536351075, 
> customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}"
> {code}
>  
> Which looks right because we can see logs from both the plugin and remote log 
> manager indicating that the remote log segment was removed.
> Now if I look on one of the leaked segment, here is what I see
>  
> {code:java}
> "2024-04-02T00:43:33.834Z","""kafka""","""10001""","[RemoteLogManager=10001 
> partition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765] Copied 
> 00000000000002971163.log to remote storage with segment-id: 
> RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765, 
> id=8dP13VDYSaiFlubl9SNBTQ}"
> "2024-04-02T00:43:33.822Z","""kafka""","""10001""","Copying log segment data 
> completed successfully, metadata: 
> RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId
> {topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765, 
> id=8dP13VDYSaiFlubl9SNBTQ}
> , startOffset=2971163, endOffset=2978396, brokerId=10001, 
> maxTimestampMs=1712010648756, eventTimestampMs=1712018599981, 
> segmentLeaderEpochs={7=2971163}, segmentSizeInBytes=459778940, 
> customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}"
> "2024-04-02T00:43:20.003Z","""kafka""","""10001""","Copying log segment data, 
> metadata: RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId
> {topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765, 
> id=8dP13VDYSaiFlubl9SNBTQ}
> , startOffset=2971163, endOffset=2978396, brokerId=10001, 
> maxTimestampMs=1712010648756, eventTimestampMs=1712018599981, 
> segmentLeaderEpochs={7=2971163}, segmentSizeInBytes=459778940, 
> customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}"
>  
> {code}
> I have no errors whatsoever indicating that the remote log deletion was 
> actually triggered and failed. 
> I tried rolling restarting my cluster to see if refreshing remote log 
> metadata from fresh state would help but that did not help. Removing segments 
> in tiered storage and restarting brokers after did not help either.
> Now if I use offset shell to get earliest and latest offsets for both 
> partitions
>  
> {code:java}
> $ /opt/kafka-3.7.0/bin/kafka-get-offsets.sh  --bootstrap-server 
> $KAFKA_CONNECT_STR --topic topic1 --partitions 764,765 -
> -time -2 
> raw_spans_datadog_3543:764:2980106
> raw_spans_datadog_3543:765:2976337
> $ $ /opt/kafka-3.7.0/bin/kafka-get-offsets.sh  --bootstrap-server 
> $KAFKA_CONNECT_STR --topic topic1 --partitions 764,765 --time -1
> raw_spans_datadog_3543:764:2980106
> raw_spans_datadog_3543:765:2978397
> {code}
> We can see that despite the retention period being largely exceeded, there 
> are 2060 offsets that are never expiring.
> One thing that might have triggered this bug was the fact that some nodes 
> were replaced with fresh disk in between the segment creation and their 
> expiration.
> The issue happened on 418 partitions out of 4513, ~9% of partitions.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to