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

Francois Visconte edited comment on KAFKA-16511 at 4/11/24 7:54 PM:
--------------------------------------------------------------------

summarising here the investigation we did with [~ckamal] 

Looking at {{__remote_log_metadata}} (infinite retention) partition that 
contains metadata for the affected partition 765 the only thing we have is:


{code:java}
partition: 27, offset: 396276, value: 
RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-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} partition: 27, 
offset: 396279, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=8dP13VDYSaiFlubl9SNBTQ}, customMetadata=Optional.empty, 
state=COPY_SEGMENT_FINISHED, eventTimestampMs=1712018613822, 
brokerId=10001}{code}

So it seems we never pushed any {{DELETE_SEGMENT_STARTED.}}


was (Author: JIRAUSER288982):
summarising here the investigation we did with [~ckamal] 

Looking at {{__remote_log_metadata}} (infinite retention) partition that 
contains metadata for the affected partition 765 the only thing we have is:


{code:java}
partition: 27, offset: 396276, value: 
RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-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} partition: 27, 
offset: 396279, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=8dP13VDYSaiFlubl9SNBTQ}, customMetadata=Optional.empty, 
state=COPY_SEGMENT_FINISHED, eventTimestampMs=1712018613822, 
brokerId=10001}{code}

So it seems we never pushed any {{DELETE_SEGMENT_STARTED.}}

> 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
>            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.



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

Reply via email to