[jira] [Comment Edited] (KAFKA-16895) RemoteCopyLagSegments metric taking active segment into account

2024-06-05 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-16895 at 6/5/24 11:48 AM:


[~chia7712] proposed a PR: https://github.com/apache/kafka/pull/16210


was (Author: JIRAUSER288982):
[~chia7712] proposed a PR

> RemoteCopyLagSegments metric taking active segment into account
> ---
>
> Key: KAFKA-16895
> URL: https://issues.apache.org/jira/browse/KAFKA-16895
> Project: Kafka
>  Issue Type: Bug
>  Components: Tiered-Storage
>Affects Versions: 3.7.0, 3.7.1
>Reporter: Francois Visconte
>Priority: Minor
>
> The RemoteCopyLagSegment is off by 1 because it also includes the active 
> segment into account while the RemoteCopyLagBytes does substract the size of 
> active segment: 
>
> {code:java}
>  long bytesLag = log.onlyLocalLogSegmentsSize() - log.activeSegment().size();
> String topic = topicIdPartition.topic();
> int partition = topicIdPartition.partition();
> long segmentsLag = log.onlyLocalLogSegmentsCount();
> brokerTopicStats.recordRemoteCopyLagBytes(topic, partition, bytesLag);
> brokerTopicStats.recordRemoteCopyLagSegments(topic, partition, segmentsLag);
> {code}



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


[jira] [Commented] (KAFKA-16895) RemoteCopyLagSegments metric taking active segment into account

2024-06-05 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16895:
---

[~chia7712] proposed a PR

> RemoteCopyLagSegments metric taking active segment into account
> ---
>
> Key: KAFKA-16895
> URL: https://issues.apache.org/jira/browse/KAFKA-16895
> Project: Kafka
>  Issue Type: Bug
>  Components: Tiered-Storage
>Affects Versions: 3.7.0, 3.7.1
>Reporter: Francois Visconte
>Priority: Minor
>
> The RemoteCopyLagSegment is off by 1 because it also includes the active 
> segment into account while the RemoteCopyLagBytes does substract the size of 
> active segment: 
>
> {code:java}
>  long bytesLag = log.onlyLocalLogSegmentsSize() - log.activeSegment().size();
> String topic = topicIdPartition.topic();
> int partition = topicIdPartition.partition();
> long segmentsLag = log.onlyLocalLogSegmentsCount();
> brokerTopicStats.recordRemoteCopyLagBytes(topic, partition, bytesLag);
> brokerTopicStats.recordRemoteCopyLagSegments(topic, partition, segmentsLag);
> {code}



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


[jira] [Created] (KAFKA-16895) RemoteCopyLagSegments metric taking active segment into account

2024-06-05 Thread Francois Visconte (Jira)
Francois Visconte created KAFKA-16895:
-

 Summary: RemoteCopyLagSegments metric taking active segment into 
account
 Key: KAFKA-16895
 URL: https://issues.apache.org/jira/browse/KAFKA-16895
 Project: Kafka
  Issue Type: Bug
  Components: Tiered-Storage
Affects Versions: 3.7.0, 3.7.1
Reporter: Francois Visconte


The RemoteCopyLagSegment is off by 1 because it also includes the active 
segment into account while the RemoteCopyLagBytes does substract the size of 
active segment: 

   
{code:java}
 long bytesLag = log.onlyLocalLogSegmentsSize() - log.activeSegment().size();
String topic = topicIdPartition.topic();
int partition = topicIdPartition.partition();
long segmentsLag = log.onlyLocalLogSegmentsCount();
brokerTopicStats.recordRemoteCopyLagBytes(topic, partition, bytesLag);
brokerTopicStats.recordRemoteCopyLagSegments(topic, partition, segmentsLag);
{code}




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


[jira] [Created] (KAFKA-16890) Failing to build aux state on broker failover

2024-06-04 Thread Francois Visconte (Jira)
Francois Visconte created KAFKA-16890:
-

 Summary: Failing to build aux state on broker failover
 Key: KAFKA-16890
 URL: https://issues.apache.org/jira/browse/KAFKA-16890
 Project: Kafka
  Issue Type: Bug
  Components: Tiered-Storage
Affects Versions: 3.7.0, 3.7.1
Reporter: Francois Visconte


We have clusters where we replace machines often falling into a state where we 
keep having "Error building remote log auxiliary state for loadtest_topic-22" 
and the partition being under-replicated until the leader is manually 
restarted. 

Looking into a specific case, here is what we observed in __remote_log_metadata 
topic:


{code:java}
 
partition: 29, offset: 183593, value: 
RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=GZeRTXLMSNe2BQjRXkg6hQ}, startOffset=10823, endOffset=11536, 
brokerId=10013, maxTimestampMs=1715774588597, eventTimestampMs=1715781657604, 
segmentLeaderEpochs={125=10823, 126=10968, 128=11047, 130=11048, 131=11324, 
133=11442, 134=11443, 135=11445, 136=11521, 137=11533, 139=11535}, 
segmentSizeInBytes=704895, customMetadata=Optional.empty, 
state=COPY_SEGMENT_STARTED}
partition: 29, offset: 183594, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=GZeRTXLMSNe2BQjRXkg6hQ}, customMetadata=Optional.empty, 
state=COPY_SEGMENT_FINISHED, eventTimestampMs=1715781658183, brokerId=10013}
partition: 29, offset: 183669, value: 
RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=L1TYzx0lQkagRIF86Kp0QQ}, startOffset=10823, endOffset=11544, 
brokerId=10008, maxTimestampMs=1715781445270, eventTimestampMs=1715782717593, 
segmentLeaderEpochs={125=10823, 126=10968, 128=11047, 130=11048, 131=11324, 
133=11442, 134=11443, 135=11445, 136=11521, 137=11533, 139=11535, 140=11537, 
142=11543}, segmentSizeInBytes=713088, customMetadata=Optional.empty, 
state=COPY_SEGMENT_STARTED}
partition: 29, offset: 183670, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=L1TYzx0lQkagRIF86Kp0QQ}, customMetadata=Optional.empty, 
state=COPY_SEGMENT_FINISHED, eventTimestampMs=1715782718370, brokerId=10008}
partition: 29, offset: 186215, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=L1TYzx0lQkagRIF86Kp0QQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1715867874617, brokerId=10008}
partition: 29, offset: 186216, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=L1TYzx0lQkagRIF86Kp0QQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1715867874725, brokerId=10008}
partition: 29, offset: 186217, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=GZeRTXLMSNe2BQjRXkg6hQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1715867874729, brokerId=10008}
partition: 29, offset: 186218, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=ClnIeN0MQsi_d4FAOFKaDA:loadtest_topic-22,
 id=GZeRTXLMSNe2BQjRXkg6hQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1715867874817, brokerId=10008}
{code}
 

It seems that at the time the leader is restarted (10013), a second copy of the 
same segment is tiered by the new leader (10008). Interestingly the segment 
doesn't have the same end offset, which is concerning. 

Then the follower sees the following error repeatedly until the leader is 
restarted: 



 
{code:java}
[2024-05-17 20:46:42,133] DEBUG [ReplicaFetcher replicaId=10013, 
leaderId=10008, fetcherId=0] Handling errors in processFetchRequest for 
partitions HashSet(loadtest_topic-22) (kafka.server.ReplicaFetcherThread)
[2024-05-17 20:46:43,174] DEBUG [ReplicaFetcher replicaId=10013, 
leaderId=10008, fetcherId=0] Received error OFFSET_MOVED_TO_TIERED_STORAGE, at 
fetch offset: 11537, topic-partition: loadtest_topic-22 
(kafka.server.ReplicaFetcherThread)
[2024-05-17 20:46:43,175] ERROR [ReplicaFetcher replicaId=10013, 
leaderId=10008, fetcherId=0] Error building remote log auxiliary state for 
loadtest_topic-22 (kafka.server.ReplicaFetcherThread)
org.apache.kafka.server.log.remote.storage.RemoteStorageException: Couldn't 
build the state from remote store for partition: loadtest_topic-22, 
currentLeaderEpoch: 153, leaderLocalLogStartOffset: 11545, 
leaderLogStartOffset: 11537, epoch: 142as the previous remote log segment 
metadata was not found
{code}
The 

[jira] [Commented] (KAFKA-16511) Leaking tiered segments

2024-05-03 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16511:
---

[~ckamal][~showuon]

I pulled the fix on one of my test cluster and I confirm it fixed the issue I 
had on some partitions. The old segments were cleaned up from remote storage 
too.

> 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 
> 02968418.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 
> 02971163.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, 

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-05-03 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-16511 at 5/3/24 3:39 PM:
---

[~ckamal] / [~showuon]

I pulled the fix on one of my test cluster and I confirm it fixed the issue I 
had on some partitions. The old segments were cleaned up from remote storage 
too.


was (Author: JIRAUSER288982):
[~ckamal][~showuon]

I pulled the fix on one of my test cluster and I confirm it fixed the issue I 
had on some partitions. The old segments were cleaned up from remote storage 
too.

> 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 
> 02968418.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 
> 02971163.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: 
> 

[jira] [Commented] (KAFKA-16511) Leaking tiered segments

2024-04-16 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16511:
---

Here is for example one of the segments and partitions where I had the issue. 

Remaining segment leaked on tiered storage that was never deleted
{code}
partition: 12, offset: 429045, value: 
RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=9481FsNMTqiXLLzIBY03lA}, startOffset=2965469, endOffset=2968297, 
brokerId=10045, maxTimestampMs=1712010650848, eventTimestampMs=1712018260978, 
segmentLeaderEpochs={7=2965469}, segmentSizeInBytes=179013013, 
customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}
partition: 12, offset: 429049, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=9481FsNMTqiXLLzIBY03lA}, customMetadata=Optional.empty, 
state=COPY_SEGMENT_FINISHED, eventTimestampMs=1712018266033, brokerId=10045}
{code}

Last events in the remote log metadata for this partition:
{code}
partition: 12, offset: 427434, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=5_3z-dusQEeSDqNC_E5ifQ}, customMetadata=Optional.empty, 
state=COPY_SEGMENT_FINISHED, eventTimestampMs=1712010391903, brokerId=10041}
partition: 12, offset: 427680, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=58Ht5YJ8SaW3JaI_lDEpsA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712012524972, brokerId=10041}
partition: 12, offset: 427681, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=58Ht5YJ8SaW3JaI_lDEpsA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712012525109, brokerId=10041}
partition: 12, offset: 428017, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=s4RCtworR-2Na8PGY6h2nQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712014486037, brokerId=10045}
partition: 12, offset: 428018, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=s4RCtworR-2Na8PGY6h2nQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712014486168, brokerId=10045}
partition: 12, offset: 428399, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=UWhaLz-GTjqPcVKL8uZOQQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712016766308, brokerId=10045}
partition: 12, offset: 428400, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=UWhaLz-GTjqPcVKL8uZOQQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712016766397, brokerId=10045}
partition: 12, offset: 429045, value: 
RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=9481FsNMTqiXLLzIBY03lA}, startOffset=2965469, endOffset=2968297, 
brokerId=10045, maxTimestampMs=1712010650848, eventTimestampMs=1712018260978, 
segmentLeaderEpochs={7=2965469}, segmentSizeInBytes=179013013, 
customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}
partition: 12, offset: 429049, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=9481FsNMTqiXLLzIBY03lA}, customMetadata=Optional.empty, 
state=COPY_SEGMENT_FINISHED, eventTimestampMs=1712018266033, brokerId=10045}
partition: 12, offset: 429265, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=TM44MmIqSKSBPmXMzjAvxA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712018873277, brokerId=10045}
partition: 12, offset: 429266, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=TM44MmIqSKSBPmXMzjAvxA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712018873411, brokerId=10045}
partition: 12, offset: 429602, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=djqwO5JeS9y2vRua-FBO3A}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712020973535, brokerId=10045}
partition: 12, offset: 429603, value: 

[jira] (KAFKA-16511) Leaking tiered segments

2024-04-16 Thread Francois Visconte (Jira)


[ https://issues.apache.org/jira/browse/KAFKA-16511 ]


Francois Visconte deleted comment on KAFKA-16511:
---

was (Author: JIRAUSER288982):
>The issue might be due to the overlapping remote log segments after a new 
>leader gets elected during rolling restart. Would you please upload the past 
>10 segments remote-log-segment metadata events for 
>5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765 partition? Thanks!

Here is 


{code:java}
partition: 27, offset: 400504, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=v8QykWvQQryMxNoS1aHMCQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712045518393, brokerId=10015}
partition: 27, offset: 400505, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=v8QykWvQQryMxNoS1aHMCQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712045518494, brokerId=10015}
partition: 27, offset: 400828, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=6K-bSNgxSnOypIrjl9OiGA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712047438507, brokerId=10015}
partition: 27, offset: 400829, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=6K-bSNgxSnOypIrjl9OiGA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712047438609, brokerId=10015}
partition: 27, offset: 401145, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=ErPfaCVhRiS6EKx7RnL1iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712049418775, brokerId=10015}
partition: 27, offset: 401146, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=ErPfaCVhRiS6EKx7RnL1iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712049418894, brokerId=10015}
partition: 27, offset: 401458, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=2yAT3R6tS3umrvU8iuMVfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712051340571, brokerId=10015}
partition: 27, offset: 401459, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=2yAT3R6tS3umrvU8iuMVfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712051340719, brokerId=10015}
partition: 27, offset: 401758, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=KLtWI-SWS7eBML87LX63SA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712053320735, brokerId=10015}
partition: 27, offset: 401759, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=KLtWI-SWS7eBML87LX63SA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712053320838, brokerId=10015}
partition: 27, offset: 539496, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=8dP13VDYSaiFlubl9SNBTQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712909568625, brokerId=10015}
partition: 27, offset: 539497, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=8dP13VDYSaiFlubl9SNBTQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712909568846, brokerId=10015}
{code}


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

[jira] (KAFKA-16511) Leaking tiered segments

2024-04-16 Thread Francois Visconte (Jira)


[ https://issues.apache.org/jira/browse/KAFKA-16511 ]


Francois Visconte deleted comment on KAFKA-16511:
---

was (Author: JIRAUSER288982):
Here is for example on one of the partition where I have the issue: 

{code}
_zIcSPWGzlqLjUTFfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712036488994, brokerId=10041}
partition: 12, offset: 432066, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=YTIk_zIcSPWGzlqLjUTFfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712036489094, brokerId=10041}
partition: 12, offset: 432319, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=8mQ_JJLfTQmkP80MYcN6Ig}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712038114012, brokerId=10041}
partition: 12, offset: 432320, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=8mQ_JJLfTQmkP80MYcN6Ig}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712038114110, brokerId=10041}
partition: 12, offset: 432593, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=oqr_-KDrSIGiGrcLQZhzvA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712039805803, brokerId=10041}
partition: 12, offset: 432594, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=oqr_-KDrSIGiGrcLQZhzvA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712039805901, brokerId=10041}
partition: 12, offset: 432914, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=OSou-FH9S4ioH5LHYUxPMg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712041682856, brokerId=10041}
partition: 12, offset: 432915, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=OSou-FH9S4ioH5LHYUxPMg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712041683023, brokerId=10041}
partition: 12, offset: 433251, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=uSPj4yS5RIO1LuBHecl7iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712043678337, brokerId=10041}
partition: 12, offset: 433252, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=uSPj4yS5RIO1LuBHecl7iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712043678474, brokerId=10041}
partition: 12, offset: 433577, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=pcCt8vc-TUyRhHOGUJBHXg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712045569214, brokerId=10041}
partition: 12, offset: 433578, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=pcCt8vc-TUyRhHOGUJBHXg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712045569333, brokerId=10041}
partition: 12, offset: 433904, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=LFqoPyg0SOiHbscnV3Ahtw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712047491475, brokerId=10041}
partition: 12, offset: 433905, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=LFqoPyg0SOiHbscnV3Ahtw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712047491576, brokerId=10041}
partition: 12, offset: 434229, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=SSyS_hbXSEKD5rgbu1S8ug}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712049443915, brokerId=10041}
partition: 12, offset: 434230, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=SSyS_hbXSEKD5rgbu1S8ug}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712049444048, brokerId=10041}
partition: 12, offset: 434584, value: 

[jira] [Commented] (KAFKA-16511) Leaking tiered segments

2024-04-16 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16511:
---

>The issue might be due to the overlapping remote log segments after a new 
>leader gets elected during rolling restart. Would you please upload the past 
>10 segments remote-log-segment metadata events for 
>5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765 partition? Thanks!

Here is 


{code:java}
partition: 27, offset: 400504, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=v8QykWvQQryMxNoS1aHMCQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712045518393, brokerId=10015}
partition: 27, offset: 400505, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=v8QykWvQQryMxNoS1aHMCQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712045518494, brokerId=10015}
partition: 27, offset: 400828, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=6K-bSNgxSnOypIrjl9OiGA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712047438507, brokerId=10015}
partition: 27, offset: 400829, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=6K-bSNgxSnOypIrjl9OiGA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712047438609, brokerId=10015}
partition: 27, offset: 401145, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=ErPfaCVhRiS6EKx7RnL1iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712049418775, brokerId=10015}
partition: 27, offset: 401146, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=ErPfaCVhRiS6EKx7RnL1iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712049418894, brokerId=10015}
partition: 27, offset: 401458, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=2yAT3R6tS3umrvU8iuMVfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712051340571, brokerId=10015}
partition: 27, offset: 401459, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=2yAT3R6tS3umrvU8iuMVfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712051340719, brokerId=10015}
partition: 27, offset: 401758, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=KLtWI-SWS7eBML87LX63SA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712053320735, brokerId=10015}
partition: 27, offset: 401759, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=KLtWI-SWS7eBML87LX63SA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712053320838, brokerId=10015}
partition: 27, offset: 539496, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=8dP13VDYSaiFlubl9SNBTQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712909568625, brokerId=10015}
partition: 27, offset: 539497, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765,
 id=8dP13VDYSaiFlubl9SNBTQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712909568846, brokerId=10015}
{code}


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

[jira] [Commented] (KAFKA-16511) Leaking tiered segments

2024-04-16 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16511:
---

Here is for example on one of the partition where I have the issue: 

{code}
_zIcSPWGzlqLjUTFfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712036488994, brokerId=10041}
partition: 12, offset: 432066, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=YTIk_zIcSPWGzlqLjUTFfw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712036489094, brokerId=10041}
partition: 12, offset: 432319, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=8mQ_JJLfTQmkP80MYcN6Ig}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712038114012, brokerId=10041}
partition: 12, offset: 432320, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=8mQ_JJLfTQmkP80MYcN6Ig}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712038114110, brokerId=10041}
partition: 12, offset: 432593, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=oqr_-KDrSIGiGrcLQZhzvA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712039805803, brokerId=10041}
partition: 12, offset: 432594, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=oqr_-KDrSIGiGrcLQZhzvA}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712039805901, brokerId=10041}
partition: 12, offset: 432914, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=OSou-FH9S4ioH5LHYUxPMg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712041682856, brokerId=10041}
partition: 12, offset: 432915, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=OSou-FH9S4ioH5LHYUxPMg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712041683023, brokerId=10041}
partition: 12, offset: 433251, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=uSPj4yS5RIO1LuBHecl7iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712043678337, brokerId=10041}
partition: 12, offset: 433252, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=uSPj4yS5RIO1LuBHecl7iQ}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712043678474, brokerId=10041}
partition: 12, offset: 433577, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=pcCt8vc-TUyRhHOGUJBHXg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712045569214, brokerId=10041}
partition: 12, offset: 433578, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=pcCt8vc-TUyRhHOGUJBHXg}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712045569333, brokerId=10041}
partition: 12, offset: 433904, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=LFqoPyg0SOiHbscnV3Ahtw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712047491475, brokerId=10041}
partition: 12, offset: 433905, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=LFqoPyg0SOiHbscnV3Ahtw}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712047491576, brokerId=10041}
partition: 12, offset: 434229, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=SSyS_hbXSEKD5rgbu1S8ug}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_STARTED, eventTimestampMs=1712049443915, brokerId=10041}
partition: 12, offset: 434230, value: 
RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic2-774,
 id=SSyS_hbXSEKD5rgbu1S8ug}, customMetadata=Optional.empty, 
state=DELETE_SEGMENT_FINISHED, eventTimestampMs=1712049444048, brokerId=10041}
partition: 12, offset: 434584, value: 

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-04-12 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-16511 at 4/12/24 8:24 AM:


I ran kafka-delete-records.sh and it did the trick


{code:java}
[RemoteLogManager=10015 
partition=5G8Ai8kBSwmQ3Ln4QRY5rA:raw_spans_datadog_3543-765] Deleted remote log 
segment RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765, 
id=8dP13VDYSaiFlubl9SNBTQ} due to leader-epoch-cache truncation. Current 
earliest-epoch-entry: EpochEntry(epoch=21, startOffset=2978397), 
segment-end-offset: 2978396 and segment-epochs: [7]
{code}

Do you happen to know what triggers the issue?



was (Author: JIRAUSER288982):
I ran kafka-delete-records.sh and it did the trick


{code:java}
[RemoteLogManager=10015 
partition=5G8Ai8kBSwmQ3Ln4QRY5rA:raw_spans_datadog_3543-765] Deleted remote log 
segment RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765, 
id=8dP13VDYSaiFlubl9SNBTQ} due to leader-epoch-cache truncation. Current 
earliest-epoch-entry: EpochEntry(epoch=21, startOffset=2978397), 
segment-end-offset: 2978396 and segment-epochs: [7]
{code}


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

[jira] [Commented] (KAFKA-16511) Leaking tiered segments

2024-04-12 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16511:
---

I ran kafka-delete-records.sh and it did the trick


{code:java}
[RemoteLogManager=10015 
partition=5G8Ai8kBSwmQ3Ln4QRY5rA:raw_spans_datadog_3543-765] Deleted remote log 
segment RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1-765, 
id=8dP13VDYSaiFlubl9SNBTQ} due to leader-epoch-cache truncation. Current 
earliest-epoch-entry: EpochEntry(epoch=21, startOffset=2978397), 
segment-end-offset: 2978396 and segment-epochs: [7]
{code}


> 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 
> 02968418.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 
> 02971163.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: 
> 

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


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

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


The content of the leader-epoch-checkpoint

{code}
$ cat leader-epoch-checkpoint 
0
2
7 2976337
21 2978397
{code}



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

(Note we already have this https://github.com/apache/kafka/pull/14766 on our 
branch)


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

(Note we already have this https://github.com/apache/kafka/pull/14766 on our 
branch)

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

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-16511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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.}}

(Note we already have this https://github.com/apache/kafka/pull/14766 on our 
branch)


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 
> 02968418.log to remote storage with segment-id: 
> RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-764, 
> id=fqGng3UURCG3-v4lETeLKQ}"
> 

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-16511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 
> 02968418.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 
> 

[jira] [Commented] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16511:
---

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

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-16511 at 4/11/24 5:26 PM:


here is what I have on the leader of partition 765
{code:java}
$ ls -l 
total 12
02978397.index
02978397.log
02978397.snapshot
02978397.timeindex
leader-epoch-checkpoint
partition.metadata
remote_log_snapshot{code}
And remote store still contains 
{code:java}
02971163-8dP13VDYSaiFlubl9SNBTQ.indexes
02971163-8dP13VDYSaiFlubl9SNBTQ.log
02971163-8dP13VDYSaiFlubl9SNBTQ.rsm-manifest{code}


[~ckamal] is there a way to unblock the situation while we find the root cause? 


was (Author: JIRAUSER288982):
here is what I have on the leader of partition 765
{code:java}
$ ls -l 
total 12
-rw-rw-r-- 1 dog dog 10485760 Apr 11 12:12 02978397.index
-rw-rw-r-- 1 dog dog        0 Apr  2 00:32 02978397.log
-rw-rw-r-- 1 dog dog       10 Apr  2 00:01 02978397.snapshot
-rw-rw-r-- 1 dog dog 10485756 Apr 11 12:12 02978397.timeindex
-rw-r--r-- 1 dog dog       25 Apr 11 12:37 leader-epoch-checkpoint
-rw-rw-r-- 1 dog dog       43 Mar 30 02:27 partition.metadata
-rw-rw-r-- 1 dog dog        0 Mar 30 02:27 remote_log_snapshot{code}
And remote store still contains 
{code:java}
02971163-8dP13VDYSaiFlubl9SNBTQ.indexes
02971163-8dP13VDYSaiFlubl9SNBTQ.log
02971163-8dP13VDYSaiFlubl9SNBTQ.rsm-manifest{code}

> 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 
> 02968418.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}
> , 

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-16511 at 4/11/24 5:25 PM:


here is what I have on the leader of partition 765
{code:java}
$ ls -l 
total 12
-rw-rw-r-- 1 dog dog 10485760 Apr 11 12:12 02978397.index
-rw-rw-r-- 1 dog dog        0 Apr  2 00:32 02978397.log
-rw-rw-r-- 1 dog dog       10 Apr  2 00:01 02978397.snapshot
-rw-rw-r-- 1 dog dog 10485756 Apr 11 12:12 02978397.timeindex
-rw-r--r-- 1 dog dog       25 Apr 11 12:37 leader-epoch-checkpoint
-rw-rw-r-- 1 dog dog       43 Mar 30 02:27 partition.metadata
-rw-rw-r-- 1 dog dog        0 Mar 30 02:27 remote_log_snapshot{code}
And remote store still contains 
{code:java}
02971163-8dP13VDYSaiFlubl9SNBTQ.indexes
02971163-8dP13VDYSaiFlubl9SNBTQ.log
02971163-8dP13VDYSaiFlubl9SNBTQ.rsm-manifest{code}


was (Author: JIRAUSER288982):
here is what I have on the leader of partition 765
{code:java}
$ ls -l 
total 12
-rw-rw-r-- 1 dog dog 10485760 Apr 11 12:12 02978397.index
-rw-rw-r-- 1 dog dog        0 Apr  2 00:32 02978397.log
-rw-rw-r-- 1 dog dog       10 Apr  2 00:01 02978397.snapshot
-rw-rw-r-- 1 dog dog 10485756 Apr 11 12:12 02978397.timeindex
-rw-r--r-- 1 dog dog       25 Apr 11 12:37 leader-epoch-checkpoint
-rw-rw-r-- 1 dog dog       43 Mar 30 02:27 partition.metadata
-rw-rw-r-- 1 dog dog        0 Mar 30 02:27 remote_log_snapshot{code}
And remote store still contains 


{code:java}
02971163-8dP13VDYSaiFlubl9SNBTQ.indexes    indexes    April 2, 
2024, 02:43:34 (UTC+02:00)    
14.7 KB
02971163-8dP13VDYSaiFlubl9SNBTQ.log    log    April 2, 2024, 
02:43:21 (UTC+02:00)    
438.5 MB
02971163-8dP13VDYSaiFlubl9SNBTQ.rsm-manifest    rsm-manifest    
April 2, 2024, 02:43:34 (UTC+02:00)    
761.0 B{code}

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

[jira] [Comment Edited] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


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

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


here is what I have on the leader of partition 765
{code:java}
$ ls -l 
total 12
-rw-rw-r-- 1 dog dog 10485760 Apr 11 12:12 02978397.index
-rw-rw-r-- 1 dog dog        0 Apr  2 00:32 02978397.log
-rw-rw-r-- 1 dog dog       10 Apr  2 00:01 02978397.snapshot
-rw-rw-r-- 1 dog dog 10485756 Apr 11 12:12 02978397.timeindex
-rw-r--r-- 1 dog dog       25 Apr 11 12:37 leader-epoch-checkpoint
-rw-rw-r-- 1 dog dog       43 Mar 30 02:27 partition.metadata
-rw-rw-r-- 1 dog dog        0 Mar 30 02:27 remote_log_snapshot{code}
And remote store still contains 


{code:java}
02971163-8dP13VDYSaiFlubl9SNBTQ.indexes    indexes    April 2, 
2024, 02:43:34 (UTC+02:00)    
14.7 KB
02971163-8dP13VDYSaiFlubl9SNBTQ.log    log    April 2, 2024, 
02:43:21 (UTC+02:00)    
438.5 MB
02971163-8dP13VDYSaiFlubl9SNBTQ.rsm-manifest    rsm-manifest    
April 2, 2024, 02:43:34 (UTC+02:00)    
761.0 B{code}


was (Author: JIRAUSER288982):
here is what I have on the leader of partition 765


{code:java}
$ ls -l 
total 12
-rw-rw-r-- 1 dog dog 10485760 Apr 11 12:12 02978397.index
-rw-rw-r-- 1 dog dog        0 Apr  2 00:32 02978397.log
-rw-rw-r-- 1 dog dog       10 Apr  2 00:01 02978397.snapshot
-rw-rw-r-- 1 dog dog 10485756 Apr 11 12:12 02978397.timeindex
-rw-r--r-- 1 dog dog       25 Apr 11 12:37 leader-epoch-checkpoint
-rw-rw-r-- 1 dog dog       43 Mar 30 02:27 partition.metadata
-rw-rw-r-- 1 dog dog        0 Mar 30 02:27 remote_log_snapshot{code}

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

[jira] [Commented] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-16511:
---

here is what I have on the leader of partition 765


{code:java}
$ ls -l 
total 12
-rw-rw-r-- 1 dog dog 10485760 Apr 11 12:12 02978397.index
-rw-rw-r-- 1 dog dog        0 Apr  2 00:32 02978397.log
-rw-rw-r-- 1 dog dog       10 Apr  2 00:01 02978397.snapshot
-rw-rw-r-- 1 dog dog 10485756 Apr 11 12:12 02978397.timeindex
-rw-r--r-- 1 dog dog       25 Apr 11 12:37 leader-epoch-checkpoint
-rw-rw-r-- 1 dog dog       43 Mar 30 02:27 partition.metadata
-rw-rw-r-- 1 dog dog        0 Mar 30 02:27 remote_log_snapshot{code}

> 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 
> 02968418.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 
> 02971163.log to remote storage with segment-id: 
> RemoteLogSegmentId{topicIdPartition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765, 
> id=8dP13VDYSaiFlubl9SNBTQ}"
> 

[jira] [Updated] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16511?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-16511:
--
Description: 
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 
02968418.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 
02971163.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 

[jira] [Updated] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16511?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-16511:
--
Labels: tiered-storage  (was: )

> 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 
> 02968418.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 
> 02971163.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: 

[jira] [Updated] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16511?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-16511:
--
Description: 
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 
02968418.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 
02971163.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 

[jira] [Created] (KAFKA-16511) Leaking tiered segments

2024-04-11 Thread Francois Visconte (Jira)
Francois Visconte created KAFKA-16511:
-

 Summary: 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


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: 

```

"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 
02968418.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}"

```

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

 

```

"2024-04-02T00:43:33.834Z","""kafka""","""10001""","[RemoteLogManager=10001 
partition=5G8Ai8kBSwmQ3Ln4QRY5rA:topic1_3543-765] Copied 
02971163.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}"

```

 

I 

[jira] [Commented] (KAFKA-15776) Update delay timeout for DelayedRemoteFetch request

2024-01-22 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15776:
---

[~ckamal] What I mean is that one of the interesting property of tiered storage 
is not having to change anything on the consumer side because the consumer 
protocol is unchanged. In our case, we have to go over every consumers to adapt 
their settings, and even with that we have suboptimal consumer performances (we 
still timeout a fraction of the fetch requests) and we pay the price of retries 
(because we are doing more S3 GETs than we should).

> Update delay timeout for DelayedRemoteFetch request
> ---
>
> Key: KAFKA-15776
> URL: https://issues.apache.org/jira/browse/KAFKA-15776
> Project: Kafka
>  Issue Type: Task
>Reporter: Kamal Chandraprakash
>Assignee: Kamal Chandraprakash
>Priority: Major
>
> We are reusing the {{fetch.max.wait.ms}} config as a delay timeout for 
> DelayedRemoteFetchPurgatory. {{fetch.max.wait.ms}} purpose is to wait for the 
> given amount of time when there is no data available to serve the FETCH 
> request.
> {code:java}
> The maximum amount of time the server will block before answering the fetch 
> request if there isn't sufficient data to immediately satisfy the requirement 
> given by fetch.min.bytes.
> {code}
> [https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala#L41]
> Using the same timeout in the DelayedRemoteFetchPurgatory can confuse the 
> user on how to configure optimal value for each purpose. Moreover, the config 
> is of *LOW* importance and most of the users won't configure it and use the 
> default value of 500 ms.
> Having the delay timeout of 500 ms in DelayedRemoteFetchPurgatory can lead to 
> higher number of expired delayed remote fetch requests when the remote 
> storage have any degradation.
> We should introduce one {{fetch.remote.max.wait.ms}} config (preferably 
> server config) to define the delay timeout for DelayedRemoteFetch requests 
> (or) take it from client similar to {{request.timeout.ms}}.



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


[jira] [Comment Edited] (KAFKA-15776) Update delay timeout for DelayedRemoteFetch request

2024-01-19 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-15776 at 1/19/24 4:52 PM:


[~ckamal] Any idea on how to move forward on that? I think having to configure 
a very high fetch.max.wait defeat the purpose of the KIP of not having to 
proceed adaptations on the consumer side.
This issue is annoying on our test environment (using s3): even with a 
fetch.max.wait of 2s we get flooded with interrupted exception in the logs 
(also probably the sign of sub-optimally cancelling fetch from tiered storage 
to eventually retry and succeed). 


was (Author: JIRAUSER288982):
[~ckamal] Any idea on how to move forward on that? I think having to configure 
a very high fetch.max.wait defeat the purpose of the KIP of not having to 
proceed adaptations on the consumer side.
This issue is annoying on our test environment (using s3) as even with a 
fetch.max.wait of 2s we get flooded with interrupted exception in the logs 
(also probably the sign of sub-optimally cancelling fetch from tiered storage 
to eventually succeed). 

> Update delay timeout for DelayedRemoteFetch request
> ---
>
> Key: KAFKA-15776
> URL: https://issues.apache.org/jira/browse/KAFKA-15776
> Project: Kafka
>  Issue Type: Task
>Reporter: Kamal Chandraprakash
>Assignee: Kamal Chandraprakash
>Priority: Major
>
> We are reusing the {{fetch.max.wait.ms}} config as a delay timeout for 
> DelayedRemoteFetchPurgatory. {{fetch.max.wait.ms}} purpose is to wait for the 
> given amount of time when there is no data available to serve the FETCH 
> request.
> {code:java}
> The maximum amount of time the server will block before answering the fetch 
> request if there isn't sufficient data to immediately satisfy the requirement 
> given by fetch.min.bytes.
> {code}
> [https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala#L41]
> Using the same timeout in the DelayedRemoteFetchPurgatory can confuse the 
> user on how to configure optimal value for each purpose. Moreover, the config 
> is of *LOW* importance and most of the users won't configure it and use the 
> default value of 500 ms.
> Having the delay timeout of 500 ms in DelayedRemoteFetchPurgatory can lead to 
> higher number of expired delayed remote fetch requests when the remote 
> storage have any degradation.
> We should introduce one {{fetch.remote.max.wait.ms}} config (preferably 
> server config) to define the delay timeout for DelayedRemoteFetch requests 
> (or) take it from client similar to {{request.timeout.ms}}.



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


[jira] [Comment Edited] (KAFKA-15776) Update delay timeout for DelayedRemoteFetch request

2024-01-19 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-15776 at 1/19/24 4:52 PM:


[~ckamal] Any idea on how to move forward on that? I think having to configure 
a very high fetch.max.wait defeat the purpose of the KIP of not having to 
proceed adaptations on the consumer side.
This issue is annoying on our test environment (using s3) as even with a 
fetch.max.wait of 2s we get flooded with interrupted exception in the logs 
(also probably the sign of sub-optimally cancelling fetch from tiered storage 
to eventually succeed). 


was (Author: JIRAUSER288982):
[~ckamal] Any idea on how to move forward on that? I think having to configure 
a very high fetch.max.wait defeat the purpose of the KIP of not having to 
proceed adaptions on the consumer side.
This issue is annoying on our test environment (using s3) as even with a 
fetch.max.wait of 2s we get flooded with interrupted exception in the logs 
(also probably the sign of sub-optimally cancelling fetch from tiered storage 
to eventually succeed). 

> Update delay timeout for DelayedRemoteFetch request
> ---
>
> Key: KAFKA-15776
> URL: https://issues.apache.org/jira/browse/KAFKA-15776
> Project: Kafka
>  Issue Type: Task
>Reporter: Kamal Chandraprakash
>Assignee: Kamal Chandraprakash
>Priority: Major
>
> We are reusing the {{fetch.max.wait.ms}} config as a delay timeout for 
> DelayedRemoteFetchPurgatory. {{fetch.max.wait.ms}} purpose is to wait for the 
> given amount of time when there is no data available to serve the FETCH 
> request.
> {code:java}
> The maximum amount of time the server will block before answering the fetch 
> request if there isn't sufficient data to immediately satisfy the requirement 
> given by fetch.min.bytes.
> {code}
> [https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala#L41]
> Using the same timeout in the DelayedRemoteFetchPurgatory can confuse the 
> user on how to configure optimal value for each purpose. Moreover, the config 
> is of *LOW* importance and most of the users won't configure it and use the 
> default value of 500 ms.
> Having the delay timeout of 500 ms in DelayedRemoteFetchPurgatory can lead to 
> higher number of expired delayed remote fetch requests when the remote 
> storage have any degradation.
> We should introduce one {{fetch.remote.max.wait.ms}} config (preferably 
> server config) to define the delay timeout for DelayedRemoteFetch requests 
> (or) take it from client similar to {{request.timeout.ms}}.



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


[jira] [Commented] (KAFKA-15776) Update delay timeout for DelayedRemoteFetch request

2024-01-19 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15776:
---

[~ckamal] Any idea on how to move forward on that? I think having to configure 
a very high fetch.max.wait defeat the purpose of the KIP of not having to 
proceed adaptions on the consumer side.
This issue is annoying on our test environment (using s3) as even with a 
fetch.max.wait of 2s we get flooded with interrupted exception in the logs 
(also probably the sign of sub-optimally cancelling fetch from tiered storage 
to eventually succeed). 

> Update delay timeout for DelayedRemoteFetch request
> ---
>
> Key: KAFKA-15776
> URL: https://issues.apache.org/jira/browse/KAFKA-15776
> Project: Kafka
>  Issue Type: Task
>Reporter: Kamal Chandraprakash
>Assignee: Kamal Chandraprakash
>Priority: Major
>
> We are reusing the {{fetch.max.wait.ms}} config as a delay timeout for 
> DelayedRemoteFetchPurgatory. {{fetch.max.wait.ms}} purpose is to wait for the 
> given amount of time when there is no data available to serve the FETCH 
> request.
> {code:java}
> The maximum amount of time the server will block before answering the fetch 
> request if there isn't sufficient data to immediately satisfy the requirement 
> given by fetch.min.bytes.
> {code}
> [https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala#L41]
> Using the same timeout in the DelayedRemoteFetchPurgatory can confuse the 
> user on how to configure optimal value for each purpose. Moreover, the config 
> is of *LOW* importance and most of the users won't configure it and use the 
> default value of 500 ms.
> Having the delay timeout of 500 ms in DelayedRemoteFetchPurgatory can lead to 
> higher number of expired delayed remote fetch requests when the remote 
> storage have any degradation.
> We should introduce one {{fetch.remote.max.wait.ms}} config (preferably 
> server config) to define the delay timeout for DelayedRemoteFetch requests 
> (or) take it from client similar to {{request.timeout.ms}}.



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


[jira] [Comment Edited] (KAFKA-15147) Measure pending and outstanding Remote Segment operations

2024-01-11 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-15147 at 1/11/24 3:35 PM:


Hi [~christo_lolov],

I tried these new metrics on a test cluster where I disrupted access to tiered 
storage (in my case s3) and: 
 * The metric is not available per topic (as opposed to documented in the KIP)
 * It's not increasing while several remote copies failed:

{code:java}
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes:Value,"0"
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments:Value,"0"{code}
Is this expected?


was (Author: JIRAUSER288982):
Hey [~christo_lolov],

I tried these new metrics on a test cluster where I disrupted access to tiered 
storage (in my case s3) and: 
 * The metric is not available per topic (as opposed to documented in the KIP)
 * It's not increasing while several remote copies failed:

{code:java}
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes:Value,"0"
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments:Value,"0"{code}
Is this expected?

> Measure pending and outstanding Remote Segment operations
> -
>
> Key: KAFKA-15147
> URL: https://issues.apache.org/jira/browse/KAFKA-15147
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
>Reporter: Jorge Esteban Quilcate Otoya
>Assignee: Christo Lolov
>Priority: Major
>  Labels: tiered-storage
> Fix For: 3.7.0
>
>
>  
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-963%3A+Upload+and+delete+lag+metrics+in+Tiered+Storage
>  
> KAFKA-15833: RemoteCopyLagBytes 
> KAFKA-16002: RemoteCopyLagSegments, RemoteDeleteLagBytes, 
> RemoteDeleteLagSegments
> KAFKA-16013: ExpiresPerSec
> KAFKA-16014: RemoteLogSizeComputationTime, RemoteLogSizeBytes, 
> RemoteLogMetadataCount
> KAFKA-15158: RemoteDeleteRequestsPerSec, RemoteDeleteErrorsPerSec, 
> BuildRemoteLogAuxStateRequestsPerSec, BuildRemoteLogAuxStateErrorsPerSec
> 
> Remote Log Segment operations (copy/delete) are executed by the Remote 
> Storage Manager, and recorded by Remote Log Metadata Manager (e.g. default 
> TopicBasedRLMM writes to the internal Kafka topic state changes on remote log 
> segments).
> As executions run, fail, and retry; it will be important to know how many 
> operations are pending and outstanding over time to alert operators.
> Pending operations are not enough to alert, as values can oscillate closer to 
> zero. An additional condition needs to apply (running time > threshold) to 
> consider an operation outstanding.
> Proposal:
> RemoteLogManager could be extended with 2 concurrent maps 
> (pendingSegmentCopies, pendingSegmentDeletes) `Map[Uuid, Long]` to measure 
> segmentId time when operation started, and based on this expose 2 metrics per 
> operation:
>  * pendingSegmentCopies: gauge of pendingSegmentCopies map
>  * outstandingSegmentCopies: loop over pending ops, and if now - startedTime 
> > timeout, then outstanding++ (maybe on debug level?)
> Is this a valuable metric to add to Tiered Storage? or better to solve on a 
> custom RLMM implementation?
> Also, does it require a KIP?
> Thanks!



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


[jira] [Comment Edited] (KAFKA-15147) Measure pending and outstanding Remote Segment operations

2024-01-11 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-15147 at 1/11/24 3:28 PM:


Hey [~christo_lolov],

I tried these new metrics on a test cluster where I disrupted access to tiered 
storage (in my case s3) and: 
 * The metric is not available per topic (as opposed to documented in the KIP)
 * It's not increasing while several remote copies failed:

{code:java}
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes:Value,"0"
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments:Value,"0"{code}
Is this expected?


was (Author: JIRAUSER288982):
I tried these new metrics on a test cluster where I disrupted access to tiered 
storage (in my case s3) and: 
 * The metric is not available per topic (as opposed to documented in the KIP)
 * It's not increasing while several remote copies failed:

{code:java}
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes:Value,"0"
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments:Value,"0"{code}
Is this expected?

> Measure pending and outstanding Remote Segment operations
> -
>
> Key: KAFKA-15147
> URL: https://issues.apache.org/jira/browse/KAFKA-15147
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
>Reporter: Jorge Esteban Quilcate Otoya
>Assignee: Christo Lolov
>Priority: Major
>  Labels: tiered-storage
> Fix For: 3.7.0
>
>
>  
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-963%3A+Upload+and+delete+lag+metrics+in+Tiered+Storage
>  
> KAFKA-15833: RemoteCopyLagBytes 
> KAFKA-16002: RemoteCopyLagSegments, RemoteDeleteLagBytes, 
> RemoteDeleteLagSegments
> KAFKA-16013: ExpiresPerSec
> KAFKA-16014: RemoteLogSizeComputationTime, RemoteLogSizeBytes, 
> RemoteLogMetadataCount
> KAFKA-15158: RemoteDeleteRequestsPerSec, RemoteDeleteErrorsPerSec, 
> BuildRemoteLogAuxStateRequestsPerSec, BuildRemoteLogAuxStateErrorsPerSec
> 
> Remote Log Segment operations (copy/delete) are executed by the Remote 
> Storage Manager, and recorded by Remote Log Metadata Manager (e.g. default 
> TopicBasedRLMM writes to the internal Kafka topic state changes on remote log 
> segments).
> As executions run, fail, and retry; it will be important to know how many 
> operations are pending and outstanding over time to alert operators.
> Pending operations are not enough to alert, as values can oscillate closer to 
> zero. An additional condition needs to apply (running time > threshold) to 
> consider an operation outstanding.
> Proposal:
> RemoteLogManager could be extended with 2 concurrent maps 
> (pendingSegmentCopies, pendingSegmentDeletes) `Map[Uuid, Long]` to measure 
> segmentId time when operation started, and based on this expose 2 metrics per 
> operation:
>  * pendingSegmentCopies: gauge of pendingSegmentCopies map
>  * outstandingSegmentCopies: loop over pending ops, and if now - startedTime 
> > timeout, then outstanding++ (maybe on debug level?)
> Is this a valuable metric to add to Tiered Storage? or better to solve on a 
> custom RLMM implementation?
> Also, does it require a KIP?
> Thanks!



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


[jira] [Comment Edited] (KAFKA-15147) Measure pending and outstanding Remote Segment operations

2024-01-11 Thread Francois Visconte (Jira)


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

Francois Visconte edited comment on KAFKA-15147 at 1/11/24 3:28 PM:


I tried these new metrics on a test cluster where I disrupted access to tiered 
storage (in my case s3) and: 
 * The metric is not available per topic (as opposed to documented in the KIP)
 * It's not increasing while several remote copies failed:

{code:java}
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes:Value,"0"
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments:Value,"0"{code}
Is this expected?


was (Author: JIRAUSER288982):
I tried these new metrics on a test cluster where I disrupted access to tiered 
storage (in my case s3) and: 
* The metric is not available per topic (as opposed to documented in the KIP)
* It's not increasing while several remote copies failed

```
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes:Value,"0"
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments:Value,"0"
```

> Measure pending and outstanding Remote Segment operations
> -
>
> Key: KAFKA-15147
> URL: https://issues.apache.org/jira/browse/KAFKA-15147
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
>Reporter: Jorge Esteban Quilcate Otoya
>Assignee: Christo Lolov
>Priority: Major
>  Labels: tiered-storage
> Fix For: 3.7.0
>
>
>  
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-963%3A+Upload+and+delete+lag+metrics+in+Tiered+Storage
>  
> KAFKA-15833: RemoteCopyLagBytes 
> KAFKA-16002: RemoteCopyLagSegments, RemoteDeleteLagBytes, 
> RemoteDeleteLagSegments
> KAFKA-16013: ExpiresPerSec
> KAFKA-16014: RemoteLogSizeComputationTime, RemoteLogSizeBytes, 
> RemoteLogMetadataCount
> KAFKA-15158: RemoteDeleteRequestsPerSec, RemoteDeleteErrorsPerSec, 
> BuildRemoteLogAuxStateRequestsPerSec, BuildRemoteLogAuxStateErrorsPerSec
> 
> Remote Log Segment operations (copy/delete) are executed by the Remote 
> Storage Manager, and recorded by Remote Log Metadata Manager (e.g. default 
> TopicBasedRLMM writes to the internal Kafka topic state changes on remote log 
> segments).
> As executions run, fail, and retry; it will be important to know how many 
> operations are pending and outstanding over time to alert operators.
> Pending operations are not enough to alert, as values can oscillate closer to 
> zero. An additional condition needs to apply (running time > threshold) to 
> consider an operation outstanding.
> Proposal:
> RemoteLogManager could be extended with 2 concurrent maps 
> (pendingSegmentCopies, pendingSegmentDeletes) `Map[Uuid, Long]` to measure 
> segmentId time when operation started, and based on this expose 2 metrics per 
> operation:
>  * pendingSegmentCopies: gauge of pendingSegmentCopies map
>  * outstandingSegmentCopies: loop over pending ops, and if now - startedTime 
> > timeout, then outstanding++ (maybe on debug level?)
> Is this a valuable metric to add to Tiered Storage? or better to solve on a 
> custom RLMM implementation?
> Also, does it require a KIP?
> Thanks!



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


[jira] [Commented] (KAFKA-15147) Measure pending and outstanding Remote Segment operations

2024-01-11 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15147:
---

I tried these new metrics on a test cluster where I disrupted access to tiered 
storage (in my case s3) and: 
* The metric is not available per topic (as opposed to documented in the KIP)
* It's not increasing while several remote copies failed

```
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagBytes:Value,"0"
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments:Value,"0"
```

> Measure pending and outstanding Remote Segment operations
> -
>
> Key: KAFKA-15147
> URL: https://issues.apache.org/jira/browse/KAFKA-15147
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
>Reporter: Jorge Esteban Quilcate Otoya
>Assignee: Christo Lolov
>Priority: Major
>  Labels: tiered-storage
> Fix For: 3.7.0
>
>
>  
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-963%3A+Upload+and+delete+lag+metrics+in+Tiered+Storage
>  
> KAFKA-15833: RemoteCopyLagBytes 
> KAFKA-16002: RemoteCopyLagSegments, RemoteDeleteLagBytes, 
> RemoteDeleteLagSegments
> KAFKA-16013: ExpiresPerSec
> KAFKA-16014: RemoteLogSizeComputationTime, RemoteLogSizeBytes, 
> RemoteLogMetadataCount
> KAFKA-15158: RemoteDeleteRequestsPerSec, RemoteDeleteErrorsPerSec, 
> BuildRemoteLogAuxStateRequestsPerSec, BuildRemoteLogAuxStateErrorsPerSec
> 
> Remote Log Segment operations (copy/delete) are executed by the Remote 
> Storage Manager, and recorded by Remote Log Metadata Manager (e.g. default 
> TopicBasedRLMM writes to the internal Kafka topic state changes on remote log 
> segments).
> As executions run, fail, and retry; it will be important to know how many 
> operations are pending and outstanding over time to alert operators.
> Pending operations are not enough to alert, as values can oscillate closer to 
> zero. An additional condition needs to apply (running time > threshold) to 
> consider an operation outstanding.
> Proposal:
> RemoteLogManager could be extended with 2 concurrent maps 
> (pendingSegmentCopies, pendingSegmentDeletes) `Map[Uuid, Long]` to measure 
> segmentId time when operation started, and based on this expose 2 metrics per 
> operation:
>  * pendingSegmentCopies: gauge of pendingSegmentCopies map
>  * outstandingSegmentCopies: loop over pending ops, and if now - startedTime 
> > timeout, then outstanding++ (maybe on debug level?)
> Is this a valuable metric to add to Tiered Storage? or better to solve on a 
> custom RLMM implementation?
> Also, does it require a KIP?
> Thanks!



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


[jira] [Updated] (KAFKA-15802) Trying to access uncopied segments metadata on listOffsets

2023-11-09 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15802:
--
Description: 
We have a tiered storage cluster running with Aiven s3 plugin. 

On our cluster, we have a process doing regular listOffsets requests. 

This triggers the following exception:
{code:java}
org.apache.kafka.common.KafkaException: 
org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException: 
Requested remote resource was not found
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$createCacheEntry$6(RemoteIndexCache.java:355)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.loadIndexFile(RemoteIndexCache.java:318)
Nov 09, 2023 1:42:01 PM com.github.benmanes.caffeine.cache.LocalAsyncCache 
lambda$handleCompletion$7
WARNING: Exception thrown during asynchronous load
java.util.concurrent.CompletionException: 
io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage S3Storage{bucketName='bucket', partSize=16777216}
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:107)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1760)
at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622)
at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165)
Caused by: io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage S3Storage{bucketName='bucket', partSize=16777216}
at io.aiven.kafka.tieredstorage.storage.s3.S3Storage.fetch(S3Storage.java:80)
at 
io.aiven.kafka.tieredstorage.manifest.SegmentManifestProvider.lambda$new$1(SegmentManifestProvider.java:59)
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:103)
... 7 more
Caused by: software.amazon.awssdk.services.s3.model.NoSuchKeyException: The 
specified key does not exist. (Service: S3, Status Code: 404, Request ID: 
CFMP27PVC9V2NNEM, Extended Request ID: 
F5qqlV06qQJ5qCuWl91oueBaha0QLMBURJudnOnFDQk+YbgFcAg70JBATcARDxN44DGo+PpfZHAsum+ioYMoOw==)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30)
at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:52)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:37)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.createCacheEntry(RemoteIndexCache.java:351)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$getIndexEntry$5(RemoteIndexCache.java:341)
at 

[jira] [Updated] (KAFKA-15802) Trying to access uncopied segments metadata on listOffsets

2023-11-09 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15802:
--
Description: 
We have a tiered storage cluster running with Aiven s3 plugin. 

On our cluster, we have a process doing regular listOffsets requests. 

This trigger a tiered storage exception:

 
{code:java}
org.apache.kafka.common.KafkaException: 
org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException: 
Requested remote resource was not found
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$createCacheEntry$6(RemoteIndexCache.java:355)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.loadIndexFile(RemoteIndexCache.java:318)
Nov 09, 2023 1:42:01 PM com.github.benmanes.caffeine.cache.LocalAsyncCache 
lambda$handleCompletion$7
WARNING: Exception thrown during asynchronous load
java.util.concurrent.CompletionException: 
io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage S3Storage{bucketName='bucket', partSize=16777216}
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:107)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1760)
at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622)
at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165)
Caused by: io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage S3Storage{bucketName='bucket', partSize=16777216}
at io.aiven.kafka.tieredstorage.storage.s3.S3Storage.fetch(S3Storage.java:80)
at 
io.aiven.kafka.tieredstorage.manifest.SegmentManifestProvider.lambda$new$1(SegmentManifestProvider.java:59)
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:103)
... 7 more
Caused by: software.amazon.awssdk.services.s3.model.NoSuchKeyException: The 
specified key does not exist. (Service: S3, Status Code: 404, Request ID: 
CFMP27PVC9V2NNEM, Extended Request ID: 
F5qqlV06qQJ5qCuWl91oueBaha0QLMBURJudnOnFDQk+YbgFcAg70JBATcARDxN44DGo+PpfZHAsum+ioYMoOw==)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30)
at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:52)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:37)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.createCacheEntry(RemoteIndexCache.java:351)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$getIndexEntry$5(RemoteIndexCache.java:341)
at 

[jira] [Updated] (KAFKA-15802) Trying to access uncopied segments metadata on listOffsets

2023-11-09 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15802:
--
Description: 
We have a tiered storage cluster running with Aiven s3 plugin. 

On our cluster, we have a process doing regular listOffsets requests. 

This trigger a tiered storage exception:

 
{code:java}
org.apache.kafka.common.KafkaException: 
org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException: 
Requested remote resource was not found
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$createCacheEntry$6(RemoteIndexCache.java:355)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.loadIndexFile(RemoteIndexCache.java:318)
Nov 09, 2023 1:42:01 PM com.github.benmanes.caffeine.cache.LocalAsyncCache 
lambda$handleCompletion$7
WARNING: Exception thrown during asynchronous load
java.util.concurrent.CompletionException: 
io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:107)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1760)
at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622)
at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165)
Caused by: io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at io.aiven.kafka.tieredstorage.storage.s3.S3Storage.fetch(S3Storage.java:80)
at 
io.aiven.kafka.tieredstorage.manifest.SegmentManifestProvider.lambda$new$1(SegmentManifestProvider.java:59)
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:103)
... 7 more
Caused by: software.amazon.awssdk.services.s3.model.NoSuchKeyException: The 
specified key does not exist. (Service: S3, Status Code: 404, Request ID: 
CFMP27PVC9V2NNEM, Extended Request ID: 
F5qqlV06qQJ5qCuWl91oueBaha0QLMBURJudnOnFDQk+YbgFcAg70JBATcARDxN44DGo+PpfZHAsum+ioYMoOw==)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30)
at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:52)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:37)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.createCacheEntry(RemoteIndexCache.java:351)
at 

[jira] [Updated] (KAFKA-15802) Trying to access uncopied segments metadata on listOffsets

2023-11-09 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15802:
--
Description: 
We have a tiered storage cluster running with Aiven s3 plugin. 

On our cluster, we have a process doing regular listOffsets requests. 

This trigger a tiered storage exception:
{panel}
org.apache.kafka.common.KafkaException: 
org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException: 
Requested remote resource was not found
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$createCacheEntry$6(RemoteIndexCache.java:355)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.loadIndexFile(RemoteIndexCache.java:318)
Nov 09, 2023 1:42:01 PM com.github.benmanes.caffeine.cache.LocalAsyncCache 
lambda$handleCompletion$7
WARNING: Exception thrown during asynchronous load
java.util.concurrent.CompletionException: 
io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage\{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:107)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1760)
at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622)
at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165)
Caused by: io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
kafka-evp-ts-988a/tiered_storage_test_normal_48e5-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage\{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at io.aiven.kafka.tieredstorage.storage.s3.S3Storage.fetch(S3Storage.java:80)
at 
io.aiven.kafka.tieredstorage.manifest.SegmentManifestProvider.lambda$new$1(SegmentManifestProvider.java:59)
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:103)
... 7 more
Caused by: software.amazon.awssdk.services.s3.model.NoSuchKeyException: The 
specified key does not exist. (Service: S3, Status Code: 404, Request ID: 
CFMP27PVC9V2NNEM, Extended Request ID: 
F5qqlV06qQJ5qCuWl91oueBaha0QLMBURJudnOnFDQk+YbgFcAg70JBATcARDxN44DGo+PpfZHAsum+ioYMoOw==)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30)
at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:52)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:37)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.createCacheEntry(RemoteIndexCache.java:351)
at 

[jira] [Updated] (KAFKA-15802) Trying to access uncopied segments metadata on listOffsets

2023-11-09 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15802:
--
Description: 
We have a tiered storage cluster running with Aiven s3 plugin. 

On our cluster, we have a process doing regular listOffsets requests. 

This trigger a tiered storage exception:
{panel}
org.apache.kafka.common.KafkaException: 
org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException: 
Requested remote resource was not found
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$createCacheEntry$6(RemoteIndexCache.java:355)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.loadIndexFile(RemoteIndexCache.java:318)
Nov 09, 2023 1:42:01 PM com.github.benmanes.caffeine.cache.LocalAsyncCache 
lambda$handleCompletion$7
WARNING: Exception thrown during asynchronous load
java.util.concurrent.CompletionException: 
io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage\{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:107)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1760)
at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622)
at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165)
Caused by: io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
cluster/topic-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage\{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at io.aiven.kafka.tieredstorage.storage.s3.S3Storage.fetch(S3Storage.java:80)
at 
io.aiven.kafka.tieredstorage.manifest.SegmentManifestProvider.lambda$new$1(SegmentManifestProvider.java:59)
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:103)
... 7 more
Caused by: software.amazon.awssdk.services.s3.model.NoSuchKeyException: The 
specified key does not exist. (Service: S3, Status Code: 404, Request ID: 
CFMP27PVC9V2NNEM, Extended Request ID: 
F5qqlV06qQJ5qCuWl91oueBaha0QLMBURJudnOnFDQk+YbgFcAg70JBATcARDxN44DGo+PpfZHAsum+ioYMoOw==)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30)
at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:52)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:37)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.createCacheEntry(RemoteIndexCache.java:351)
at 

[jira] [Created] (KAFKA-15802) Trying to access uncopied segments metadata on listOffsets

2023-11-09 Thread Francois Visconte (Jira)
Francois Visconte created KAFKA-15802:
-

 Summary: Trying to access uncopied segments metadata on listOffsets
 Key: KAFKA-15802
 URL: https://issues.apache.org/jira/browse/KAFKA-15802
 Project: Kafka
  Issue Type: Bug
  Components: Tiered-Storage
Affects Versions: 3.6.0
Reporter: Francois Visconte


We have a tiered storage cluster running with Aiven s3 plugin. 

On our cluster, we have a process doing regular listOffsets requests. 

This trigger a tiered storage exception:
{panel}
org.apache.kafka.common.KafkaException: 
org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException: 
Requested remote resource was not found
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.lambda$createCacheEntry$6(RemoteIndexCache.java:355)
at 
org.apache.kafka.storage.internals.log.RemoteIndexCache.loadIndexFile(RemoteIndexCache.java:318)
Nov 09, 2023 1:42:01 PM com.github.benmanes.caffeine.cache.LocalAsyncCache 
lambda$handleCompletion$7
WARNING: Exception thrown during asynchronous load
java.util.concurrent.CompletionException: 
io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
kafka-evp-ts-988a/tiered_storage_test_normal_48e5-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage\{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:107)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
at 
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.exec(CompletableFuture.java:1760)
at 
java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
at 
java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
at 
java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622)
at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165)
Caused by: io.aiven.kafka.tieredstorage.storage.KeyNotFoundException: Key 
kafka-evp-ts-988a/tiered_storage_test_normal_48e5-0A_3phS5QWu9eU28KG0Lxg/24/00149691-Rdf4cUR_S4OYAGImco6Lbg.rsm-manifest
 does not exists in storage 
S3Storage\{bucketName='dd-kafka-tiered-storage-staging-us1-staging-dog', 
partSize=16777216}
at 
io.aiven.kafka.tieredstorage.storage.s3.S3Storage.fetch(S3Storage.java:80)
at 
io.aiven.kafka.tieredstorage.manifest.SegmentManifestProvider.lambda$new$1(SegmentManifestProvider.java:59)
at 
com.github.benmanes.caffeine.cache.CacheLoader.lambda$asyncLoad$0(CacheLoader.java:103)
... 7 more
Caused by: software.amazon.awssdk.services.s3.model.NoSuchKeyException: The 
specified key does not exist. (Service: S3, Status Code: 404, Request ID: 
CFMP27PVC9V2NNEM, Extended Request ID: 
F5qqlV06qQJ5qCuWl91oueBaha0QLMBURJudnOnFDQk+YbgFcAg70JBATcARDxN44DGo+PpfZHAsum+ioYMoOw==)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30)
at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:52)
at 

[jira] [Commented] (KAFKA-15525) Segment uploads stop working following a broker failure

2023-10-02 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15525:
---

I was planning to do that change anyways but the strange thing is that it 
doesn't happen consistently when I'm shutting down brokers in my cluster. 

> Segment uploads stop working following a broker failure
> ---
>
> Key: KAFKA-15525
> URL: https://issues.apache.org/jira/browse/KAFKA-15525
> Project: Kafka
>  Issue Type: Bug
>  Components: Tiered-Storage
>Affects Versions: 3.6.0
>Reporter: Francois Visconte
>Priority: Major
>
> I have a tiered-storage enabled cluster and topic where I continuously 
> produce and consume to/from a TS-enabled topic on that cluster.
> Here are the topic settings I’m using: 
> {code:java}
> local.retention.ms=12
> remote.storage.enable=true
> retention.ms: 1080
> segment.bytes: 51200
> {code}
> Here are my broker settings:
> {code:java}
> remote.log.storage.system.enable=true
> remote.log.storage.manager.class.path=/opt/kafka/tiered-storage-libs/*
> remote.log.storage.manager.class.name=io.aiven.kafka.tieredstorage.RemoteStorageManager
> remote.log.metadata.manager.class.name=org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager
> remote.log.metadata.manager.listener.name=INTERNAL_PLAINTEXT
> remote.log.manager.task.interval.ms=5000
> remote.log.manager.thread.pool.size=10
> remote.log.reader.threads=10
> remote.log.reader.max.pending.tasks=100
> rlmm.config.remote.log.metadata.topic.replication.factor=1
> rlmm.config.remote.log.metadata.topic.num.partitions=50
> rlmm.config.remote.log.metadata.topic.retention.ms=-1
> rsm.config.chunk.cache.class=io.aiven.kafka.tieredstorage.chunkmanager.cache.DiskBasedChunkCache
> rsm.config.chunk.cache.path=/data/tiered-storage-cache
> rsm.config.chunk.cache.size=1073741824
> rsm.config.metrics.recording.level=DEBUG    
> rsm.config.storage.aws.credentials.provider.class=software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider
> rsm.config.storage.backend.class.name=io.aiven.kafka.tieredstorage.storage.s3.S3Storage
> rsm.config.storage.backend.class=io.aiven.kafka.tieredstorage.storage.s3.S3Storage
> rsm.config.storage.s3.region=us-east-1
> rsm.config.chunk.size=102400
> rsm.config.storage.s3.multipart.upload.part.size=16777216 {code}
> When a broker in the cluster get rotated (replaced or restarted) some brokers 
> start throwing this error repeatedly: 
> {code:java}
> [RemoteLogManager=1 partition=yTypIvtBRY2l3sD4-8M7fA:loadgen-3] Error 
> occurred while copying log segments of partition: 
> yTypIvtBRY2l3sD4-8M7fA:loadgen-3 
> java.util.concurrent.ExecutionException: 
> org.apache.kafka.common.KafkaException: 
> java.util.concurrent.TimeoutException: Timed out in catching up with the 
> expected offset by consumer.
>     at 
> java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
>     at 
> java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2073)
>     at 
> kafka.log.remote.RemoteLogManager$RLMTask.copyLogSegment(RemoteLogManager.java:728)
>     at 
> kafka.log.remote.RemoteLogManager$RLMTask.copyLogSegmentsToRemote(RemoteLogManager.java:687)
>     at 
> kafka.log.remote.RemoteLogManager$RLMTask.run(RemoteLogManager.java:790)
>     at 
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
>     at 
> java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305)
>     at 
> java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305)
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
>     at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
>     at java.base/java.lang.Thread.run(Thread.java:833)
> Caused by: org.apache.kafka.common.KafkaException: 
> java.util.concurrent.TimeoutException: Timed out in catching up with the 
> expected offset by consumer.
>     at 
> org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.lambda$storeRemoteLogMetadata$0(TopicBasedRemoteLogMetadataManager.java:188)
>     at 
> java.base/java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:718)
>     at 
> java.base/java.util.concurrent.CompletableFuture$Completion.exec(CompletableFuture.java:483)
>     at 
> java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
>     at 
> java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
>     at 
> java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
>     at 
> 

[jira] [Created] (KAFKA-15525) Segment uploads stop working following a broker failure

2023-10-02 Thread Francois Visconte (Jira)
Francois Visconte created KAFKA-15525:
-

 Summary: Segment uploads stop working following a broker failure
 Key: KAFKA-15525
 URL: https://issues.apache.org/jira/browse/KAFKA-15525
 Project: Kafka
  Issue Type: Bug
  Components: Tiered-Storage
Affects Versions: 3.6.0
Reporter: Francois Visconte


I have a tiered-storage enabled cluster and topic where I continuously produce 
and consume to/from a TS-enabled topic on that cluster.

Here are the topic settings I’m using: 

{code:java}
local.retention.ms=12
remote.storage.enable=true
retention.ms: 1080
segment.bytes: 51200
{code}
Here are my broker settings:
{code:java}
remote.log.storage.system.enable=true
remote.log.storage.manager.class.path=/opt/kafka/tiered-storage-libs/*
remote.log.storage.manager.class.name=io.aiven.kafka.tieredstorage.RemoteStorageManager
remote.log.metadata.manager.class.name=org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager
remote.log.metadata.manager.listener.name=INTERNAL_PLAINTEXT
remote.log.manager.task.interval.ms=5000
remote.log.manager.thread.pool.size=10
remote.log.reader.threads=10
remote.log.reader.max.pending.tasks=100
rlmm.config.remote.log.metadata.topic.replication.factor=1
rlmm.config.remote.log.metadata.topic.num.partitions=50
rlmm.config.remote.log.metadata.topic.retention.ms=-1
rsm.config.chunk.cache.class=io.aiven.kafka.tieredstorage.chunkmanager.cache.DiskBasedChunkCache
rsm.config.chunk.cache.path=/data/tiered-storage-cache
rsm.config.chunk.cache.size=1073741824
rsm.config.metrics.recording.level=DEBUG    
rsm.config.storage.aws.credentials.provider.class=software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider
rsm.config.storage.backend.class.name=io.aiven.kafka.tieredstorage.storage.s3.S3Storage
rsm.config.storage.backend.class=io.aiven.kafka.tieredstorage.storage.s3.S3Storage
rsm.config.storage.s3.region=us-east-1
rsm.config.chunk.size=102400
rsm.config.storage.s3.multipart.upload.part.size=16777216 {code}
When a broker in the cluster get rotated (replaced or restarted) some brokers 
start throwing this error repeatedly: 
{code:java}
[RemoteLogManager=1 partition=yTypIvtBRY2l3sD4-8M7fA:loadgen-3] Error 
occurred while copying log segments of partition: 
yTypIvtBRY2l3sD4-8M7fA:loadgen-3 

java.util.concurrent.ExecutionException: 
org.apache.kafka.common.KafkaException: java.util.concurrent.TimeoutException: 
Timed out in catching up with the expected offset by consumer.
    at 
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
    at 
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2073)
    at 
kafka.log.remote.RemoteLogManager$RLMTask.copyLogSegment(RemoteLogManager.java:728)
    at 
kafka.log.remote.RemoteLogManager$RLMTask.copyLogSegmentsToRemote(RemoteLogManager.java:687)
    at kafka.log.remote.RemoteLogManager$RLMTask.run(RemoteLogManager.java:790)
    at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
    at 
java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305)
    at 
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: org.apache.kafka.common.KafkaException: 
java.util.concurrent.TimeoutException: Timed out in catching up with the 
expected offset by consumer.
    at 
org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager.lambda$storeRemoteLogMetadata$0(TopicBasedRemoteLogMetadataManager.java:188)
    at 
java.base/java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:718)
    at 
java.base/java.util.concurrent.CompletableFuture$Completion.exec(CompletableFuture.java:483)
    at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373)
    at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182)
    at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655)
    at 
java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622)
    at 
java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165)
Caused by: java.util.concurrent.TimeoutException: Timed out in catching up with 
the expected offset by consumer.
    at 
org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager.waitTillConsumptionCatchesUp(ConsumerManager.java:121)
    at 
org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager.waitTillConsumptionCatchesUp(ConsumerManager.java:89)
    at 

[jira] [Commented] (KAFKA-15414) remote logs get deleted after partition reassignment

2023-09-22 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15414:
---

[~showuon] It's working now with the latest 3.6 branch. Thanks

> remote logs get deleted after partition reassignment
> 
>
> Key: KAFKA-15414
> URL: https://issues.apache.org/jira/browse/KAFKA-15414
> Project: Kafka
>  Issue Type: Bug
>Reporter: Luke Chen
>Assignee: Kamal Chandraprakash
>Priority: Blocker
> Fix For: 3.6.0
>
> Attachments: Screenshot 2023-09-12 at 13.53.07.png, 
> image-2023-08-29-11-12-58-875.png
>
>
> it seems I'm reaching that codepath when running reassignments on my cluster 
> and segment are deleted from remote store despite a huge retention (topic 
> created a few hours ago with 1000h retention).
> It seems to happen consistently on some partitions when reassigning but not 
> all partitions.
> My test:
> I have a test topic with 30 partition configured with 1000h global retention 
> and 2 minutes local retention
> I have a load tester producing to all partitions evenly
> I have consumer load tester consuming that topic
> I regularly reset offsets to earliest on my consumer to test backfilling from 
> tiered storage.
> My consumer was catching up consuming the backlog and I wanted to upscale my 
> cluster to speed up recovery: I upscaled my cluster from 3 to 12 brokers and 
> reassigned my test topic to all available brokers to have an even 
> leader/follower count per broker.
> When I triggered the reassignment, the consumer lag dropped on some of my 
> topic partitions:
> !image-2023-08-29-11-12-58-875.png|width=800,height=79! Screenshot 2023-08-28 
> at 20 57 09
> Later I tried to reassign back my topic to 3 brokers and the issue happened 
> again.
> Both times in my logs, I've seen a bunch of logs like:
> [RemoteLogManager=10005 partition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17] 
> Deleted remote log segment RemoteLogSegmentId
> {topicIdPartition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17, 
> id=Mk0chBQrTyKETTawIulQog}
> due to leader epoch cache truncation. Current earliest epoch: 
> EpochEntry(epoch=14, startOffset=46776780), segmentEndOffset: 46437796 and 
> segmentEpochs: [10]
> Looking at my s3 bucket. The segments prior to my reassignment have been 
> indeed deleted.



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


[jira] [Created] (KAFKA-15477) Kafka won't shutdown when deleting remote segment

2023-09-19 Thread Francois Visconte (Jira)
Francois Visconte created KAFKA-15477:
-

 Summary: Kafka won't shutdown when deleting remote segment
 Key: KAFKA-15477
 URL: https://issues.apache.org/jira/browse/KAFKA-15477
 Project: Kafka
  Issue Type: Bug
  Components: core
Affects Versions: 3.6.0
Reporter: Francois Visconte


When brokers are busy deleting a bunch of segments (following a topic removal 
using tiered storage), brokers won't respond to sigterm signal and cleanly 
shutdown. 
Intead, they keep removing remote segment until it's fully completed (which can 
take time for topics with long retention).



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


[jira] [Commented] (KAFKA-15477) Kafka won't shutdown when deleting remote segments

2023-09-19 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15477:
---

cc [~satishd] 

> Kafka won't shutdown when deleting remote segments
> --
>
> Key: KAFKA-15477
> URL: https://issues.apache.org/jira/browse/KAFKA-15477
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 3.6.0
>Reporter: Francois Visconte
>Priority: Minor
>
> When brokers are busy deleting a bunch of segments (following a topic removal 
> using tiered storage), brokers won't respond to sigterm signal and cleanly 
> shutdown. 
> Intead, they keep removing remote segment until it's fully completed (which 
> can take time for topics with long retention).



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


[jira] [Updated] (KAFKA-15477) Kafka won't shutdown when deleting remote segments

2023-09-19 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15477:
--
Summary: Kafka won't shutdown when deleting remote segments  (was: Kafka 
won't shutdown when deleting remote segment)

> Kafka won't shutdown when deleting remote segments
> --
>
> Key: KAFKA-15477
> URL: https://issues.apache.org/jira/browse/KAFKA-15477
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 3.6.0
>Reporter: Francois Visconte
>Priority: Minor
>
> When brokers are busy deleting a bunch of segments (following a topic removal 
> using tiered storage), brokers won't respond to sigterm signal and cleanly 
> shutdown. 
> Intead, they keep removing remote segment until it's fully completed (which 
> can take time for topics with long retention).



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


[jira] [Updated] (KAFKA-15414) remote logs get deleted after partition reassignment

2023-09-12 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15414?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15414:
--
Attachment: Screenshot 2023-09-12 at 13.53.07.png

> remote logs get deleted after partition reassignment
> 
>
> Key: KAFKA-15414
> URL: https://issues.apache.org/jira/browse/KAFKA-15414
> Project: Kafka
>  Issue Type: Bug
>Reporter: Luke Chen
>Assignee: Kamal Chandraprakash
>Priority: Blocker
> Fix For: 3.6.0
>
> Attachments: Screenshot 2023-09-12 at 13.53.07.png, 
> image-2023-08-29-11-12-58-875.png
>
>
> it seems I'm reaching that codepath when running reassignments on my cluster 
> and segment are deleted from remote store despite a huge retention (topic 
> created a few hours ago with 1000h retention).
> It seems to happen consistently on some partitions when reassigning but not 
> all partitions.
> My test:
> I have a test topic with 30 partition configured with 1000h global retention 
> and 2 minutes local retention
> I have a load tester producing to all partitions evenly
> I have consumer load tester consuming that topic
> I regularly reset offsets to earliest on my consumer to test backfilling from 
> tiered storage.
> My consumer was catching up consuming the backlog and I wanted to upscale my 
> cluster to speed up recovery: I upscaled my cluster from 3 to 12 brokers and 
> reassigned my test topic to all available brokers to have an even 
> leader/follower count per broker.
> When I triggered the reassignment, the consumer lag dropped on some of my 
> topic partitions:
> !image-2023-08-29-11-12-58-875.png|width=800,height=79! Screenshot 2023-08-28 
> at 20 57 09
> Later I tried to reassign back my topic to 3 brokers and the issue happened 
> again.
> Both times in my logs, I've seen a bunch of logs like:
> [RemoteLogManager=10005 partition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17] 
> Deleted remote log segment RemoteLogSegmentId
> {topicIdPartition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17, 
> id=Mk0chBQrTyKETTawIulQog}
> due to leader epoch cache truncation. Current earliest epoch: 
> EpochEntry(epoch=14, startOffset=46776780), segmentEndOffset: 46437796 and 
> segmentEpochs: [10]
> Looking at my s3 bucket. The segments prior to my reassignment have been 
> indeed deleted.



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


[jira] [Commented] (KAFKA-15414) remote logs get deleted after partition reassignment

2023-09-12 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15414:
---

Not sure it's the same issue happening again but I have a strange behaviour 
while trying to reassign my partitions while consuming from the past (and 
hitting tiered-storage).

It seems that at some point my consumer offset lag is going backward 
!Screenshot 2023-09-12 at 13.53.07.png|width=1355,height=191!
And I have a burst of errors like on a handful of partitions (3 partitions out 
of 32)


{code:java}
[ReplicaFetcher replicaId=10002, leaderId=10007, fetcherId=2] Error building 
remote log auxiliary state for loadtest14-21 
org.apache.kafka.server.log.remote.storage.RemoteStorageException: Couldn't 
build the state from remote store for partition: loadtest14-21, 
currentLeaderEpoch: 13, leaderLocalLogStartOffset: 81012034, 
leaderLogStartOffset: 0, epoch: 12as the previous remote log segment metadata 
was not found
    at 
kafka.server.ReplicaFetcherTierStateMachine.buildRemoteLogAuxState(ReplicaFetcherTierStateMachine.java:252)
    at 
kafka.server.ReplicaFetcherTierStateMachine.start(ReplicaFetcherTierStateMachine.java:102)
    at 
kafka.server.AbstractFetcherThread.handleOffsetsMovedToTieredStorage(AbstractFetcherThread.scala:761)
    at 
kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:412)
    at scala.Option.foreach(Option.scala:437)
    at 
kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:332)
    at 
kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:331)
    at 
kafka.utils.Implicits$MapExtensionMethods$.$anonfun$forKeyValue$1(Implicits.scala:62)
    at 
scala.collection.convert.JavaCollectionWrappers$JMapWrapperLike.foreachEntry(JavaCollectionWrappers.scala:407)
    at 
scala.collection.convert.JavaCollectionWrappers$JMapWrapperLike.foreachEntry$(JavaCollectionWrappers.scala:403)
    at 
scala.collection.convert.JavaCollectionWrappers$AbstractJMapWrapper.foreachEntry(JavaCollectionWrappers.scala:321)
    at 
kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:331)
    at 
kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:130)
    at 
kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:129)
    at scala.Option.foreach(Option.scala:437)
    at 
kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:129)
    at 
kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112)
    at kafka.server.ReplicaFetcherThread.doWork(ReplicaFetcherThread.scala:98)
    at 
org.apache.kafka.server.util.ShutdownableThread.run(ShutdownableThread.java:130)

{code}
 

 

> remote logs get deleted after partition reassignment
> 
>
> Key: KAFKA-15414
> URL: https://issues.apache.org/jira/browse/KAFKA-15414
> Project: Kafka
>  Issue Type: Bug
>Reporter: Luke Chen
>Assignee: Kamal Chandraprakash
>Priority: Blocker
> Fix For: 3.6.0
>
> Attachments: Screenshot 2023-09-12 at 13.53.07.png, 
> image-2023-08-29-11-12-58-875.png
>
>
> it seems I'm reaching that codepath when running reassignments on my cluster 
> and segment are deleted from remote store despite a huge retention (topic 
> created a few hours ago with 1000h retention).
> It seems to happen consistently on some partitions when reassigning but not 
> all partitions.
> My test:
> I have a test topic with 30 partition configured with 1000h global retention 
> and 2 minutes local retention
> I have a load tester producing to all partitions evenly
> I have consumer load tester consuming that topic
> I regularly reset offsets to earliest on my consumer to test backfilling from 
> tiered storage.
> My consumer was catching up consuming the backlog and I wanted to upscale my 
> cluster to speed up recovery: I upscaled my cluster from 3 to 12 brokers and 
> reassigned my test topic to all available brokers to have an even 
> leader/follower count per broker.
> When I triggered the reassignment, the consumer lag dropped on some of my 
> topic partitions:
> !image-2023-08-29-11-12-58-875.png|width=800,height=79! Screenshot 2023-08-28 
> at 20 57 09
> Later I tried to reassign back my topic to 3 brokers and the issue happened 
> again.
> Both times in my logs, I've seen a bunch of logs like:
> [RemoteLogManager=10005 partition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17] 
> Deleted remote log segment RemoteLogSegmentId
> {topicIdPartition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17, 
> id=Mk0chBQrTyKETTawIulQog}
> due to leader epoch cache truncation. Current earliest epoch: 
> EpochEntry(epoch=14, 

[jira] [Updated] (KAFKA-15414) remote logs get deleted after partition reassignment

2023-09-12 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15414?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15414:
--
Attachment: (was: Screenshot 2023-09-12 at 13.53.07.png)

> remote logs get deleted after partition reassignment
> 
>
> Key: KAFKA-15414
> URL: https://issues.apache.org/jira/browse/KAFKA-15414
> Project: Kafka
>  Issue Type: Bug
>Reporter: Luke Chen
>Assignee: Kamal Chandraprakash
>Priority: Blocker
> Fix For: 3.6.0
>
> Attachments: Screenshot 2023-09-12 at 13.53.07.png, 
> image-2023-08-29-11-12-58-875.png
>
>
> it seems I'm reaching that codepath when running reassignments on my cluster 
> and segment are deleted from remote store despite a huge retention (topic 
> created a few hours ago with 1000h retention).
> It seems to happen consistently on some partitions when reassigning but not 
> all partitions.
> My test:
> I have a test topic with 30 partition configured with 1000h global retention 
> and 2 minutes local retention
> I have a load tester producing to all partitions evenly
> I have consumer load tester consuming that topic
> I regularly reset offsets to earliest on my consumer to test backfilling from 
> tiered storage.
> My consumer was catching up consuming the backlog and I wanted to upscale my 
> cluster to speed up recovery: I upscaled my cluster from 3 to 12 brokers and 
> reassigned my test topic to all available brokers to have an even 
> leader/follower count per broker.
> When I triggered the reassignment, the consumer lag dropped on some of my 
> topic partitions:
> !image-2023-08-29-11-12-58-875.png|width=800,height=79! Screenshot 2023-08-28 
> at 20 57 09
> Later I tried to reassign back my topic to 3 brokers and the issue happened 
> again.
> Both times in my logs, I've seen a bunch of logs like:
> [RemoteLogManager=10005 partition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17] 
> Deleted remote log segment RemoteLogSegmentId
> {topicIdPartition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17, 
> id=Mk0chBQrTyKETTawIulQog}
> due to leader epoch cache truncation. Current earliest epoch: 
> EpochEntry(epoch=14, startOffset=46776780), segmentEndOffset: 46437796 and 
> segmentEpochs: [10]
> Looking at my s3 bucket. The segments prior to my reassignment have been 
> indeed deleted.



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


[jira] [Updated] (KAFKA-15414) remote logs get deleted after partition reassignment

2023-09-12 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15414?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-15414:
--
Attachment: Screenshot 2023-09-12 at 13.53.07.png

> remote logs get deleted after partition reassignment
> 
>
> Key: KAFKA-15414
> URL: https://issues.apache.org/jira/browse/KAFKA-15414
> Project: Kafka
>  Issue Type: Bug
>Reporter: Luke Chen
>Assignee: Kamal Chandraprakash
>Priority: Blocker
> Fix For: 3.6.0
>
> Attachments: Screenshot 2023-09-12 at 13.53.07.png, 
> image-2023-08-29-11-12-58-875.png
>
>
> it seems I'm reaching that codepath when running reassignments on my cluster 
> and segment are deleted from remote store despite a huge retention (topic 
> created a few hours ago with 1000h retention).
> It seems to happen consistently on some partitions when reassigning but not 
> all partitions.
> My test:
> I have a test topic with 30 partition configured with 1000h global retention 
> and 2 minutes local retention
> I have a load tester producing to all partitions evenly
> I have consumer load tester consuming that topic
> I regularly reset offsets to earliest on my consumer to test backfilling from 
> tiered storage.
> My consumer was catching up consuming the backlog and I wanted to upscale my 
> cluster to speed up recovery: I upscaled my cluster from 3 to 12 brokers and 
> reassigned my test topic to all available brokers to have an even 
> leader/follower count per broker.
> When I triggered the reassignment, the consumer lag dropped on some of my 
> topic partitions:
> !image-2023-08-29-11-12-58-875.png|width=800,height=79! Screenshot 2023-08-28 
> at 20 57 09
> Later I tried to reassign back my topic to 3 brokers and the issue happened 
> again.
> Both times in my logs, I've seen a bunch of logs like:
> [RemoteLogManager=10005 partition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17] 
> Deleted remote log segment RemoteLogSegmentId
> {topicIdPartition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17, 
> id=Mk0chBQrTyKETTawIulQog}
> due to leader epoch cache truncation. Current earliest epoch: 
> EpochEntry(epoch=14, startOffset=46776780), segmentEndOffset: 46437796 and 
> segmentEpochs: [10]
> Looking at my s3 bucket. The segments prior to my reassignment have been 
> indeed deleted.



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


[jira] [Resolved] (KAFKA-13872) Partitions are truncated when leader is replaced

2023-08-31 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte resolved KAFKA-13872.
---
Resolution: Won't Fix

transitioning to won't fix as this seems the expected behaviour.

> Partitions are truncated when leader is replaced
> 
>
> Key: KAFKA-13872
> URL: https://issues.apache.org/jira/browse/KAFKA-13872
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.7.2
>Reporter: Francois Visconte
>Priority: Major
> Attachments: extract-2022-05-04T15_50_34.110Z.csv
>
>
> Sample setup:
>  * a topic with one partition and RF=3
>  * a producer using acks=1
>  * min.insync.replicas to 1
>  * 3 brokers 1,2,3
>  * Preferred leader of the partition is brokerId 0
>  
> Steps to reproduce the issue
>  * Producer keeps producing to the partition, leader is brokerId=0
>  * At some point, replicas 1 and 2 are falling behind and removed from the ISR
>  * The leader broker 0 has an hardware failure
>  * Partition transition to offline
>  * This leader is replaced with a new broker with an empty disk and the same 
> broker id 0
>  * Partition transition from offline to online with leader 0, ISR = 0
>  * Followers see the leader offset is 0 and decide to truncate their 
> partitions to 0, ISR=0,1,2
>  * At this point all the topic data has been removed from all replicas and 
> partition size drops to 0 on all replicas
> Attached some of the relevant logs. I can provide more logs if necessary



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


[jira] [Commented] (KAFKA-15414) remote logs get deleted after partition reassignment

2023-08-31 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-15414:
---

[~satish.duggana] let me know if you want me to try out my initial test with 
some ongoing work.

> remote logs get deleted after partition reassignment
> 
>
> Key: KAFKA-15414
> URL: https://issues.apache.org/jira/browse/KAFKA-15414
> Project: Kafka
>  Issue Type: Bug
>Reporter: Luke Chen
>Priority: Blocker
> Fix For: 3.6.0
>
> Attachments: image-2023-08-29-11-12-58-875.png
>
>
> it seems I'm reaching that codepath when running reassignments on my cluster 
> and segment are deleted from remote store despite a huge retention (topic 
> created a few hours ago with 1000h retention).
> It seems to happen consistently on some partitions when reassigning but not 
> all partitions.
> My test:
> I have a test topic with 30 partition configured with 1000h global retention 
> and 2 minutes local retention
> I have a load tester producing to all partitions evenly
> I have consumer load tester consuming that topic
> I regularly reset offsets to earliest on my consumer to test backfilling from 
> tiered storage.
> My consumer was catching up consuming the backlog and I wanted to upscale my 
> cluster to speed up recovery: I upscaled my cluster from 3 to 12 brokers and 
> reassigned my test topic to all available brokers to have an even 
> leader/follower count per broker.
> When I triggered the reassignment, the consumer lag dropped on some of my 
> topic partitions:
> !image-2023-08-29-11-12-58-875.png|width=800,height=79! Screenshot 2023-08-28 
> at 20 57 09
> Later I tried to reassign back my topic to 3 brokers and the issue happened 
> again.
> Both times in my logs, I've seen a bunch of logs like:
> [RemoteLogManager=10005 partition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17] 
> Deleted remote log segment RemoteLogSegmentId
> {topicIdPartition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17, 
> id=Mk0chBQrTyKETTawIulQog}
> due to leader epoch cache truncation. Current earliest epoch: 
> EpochEntry(epoch=14, startOffset=46776780), segmentEndOffset: 46437796 and 
> segmentEpochs: [10]
> Looking at my s3 bucket. The segments prior to my reassignment have been 
> indeed deleted.



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


[jira] [Commented] (KAFKA-13872) Partitions are truncated when leader is replaced

2022-05-05 Thread Francois Visconte (Jira)


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

Francois Visconte commented on KAFKA-13872:
---

I was expecting the partition to stay offline until I decide to set the 
unclean.leader.election to true and one of the lagging replica to take ownership

> Partitions are truncated when leader is replaced
> 
>
> Key: KAFKA-13872
> URL: https://issues.apache.org/jira/browse/KAFKA-13872
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.7.2
>Reporter: Francois Visconte
>Priority: Major
> Attachments: extract-2022-05-04T15_50_34.110Z.csv
>
>
> Sample setup:
>  * a topic with one partition and RF=3
>  * a producer using acks=1
>  * min.insync.replicas to 1
>  * 3 brokers 1,2,3
>  * Preferred leader of the partition is brokerId 0
>  
> Steps to reproduce the issue
>  * Producer keeps producing to the partition, leader is brokerId=0
>  * At some point, replicas 1 and 2 are falling behind and removed from the ISR
>  * The leader broker 0 has an hardware failure
>  * Partition transition to offline
>  * This leader is replaced with a new broker with an empty disk and the same 
> broker id 0
>  * Partition transition from offline to online with leader 0, ISR = 0
>  * Followers see the leader offset is 0 and decide to truncate their 
> partitions to 0, ISR=0,1,2
>  * At this point all the topic data has been removed from all replicas and 
> partition size drops to 0 on all replicas
> Attached some of the relevant logs. I can provide more logs if necessary



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Updated] (KAFKA-13872) Partitions are truncated when leader is replaced

2022-05-04 Thread Francois Visconte (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Francois Visconte updated KAFKA-13872:
--
Description: 
Sample setup:
 * a topic with one partition and RF=3
 * a producer using acks=1
 * min.insync.replicas to 1
 * 3 brokers 1,2,3
 * Preferred leader of the partition is brokerId 0

 

Steps to reproduce the issue
 * Producer keeps producing to the partition, leader is brokerId=0
 * At some point, replicas 1 and 2 are falling behind and removed from the ISR
 * The leader broker 0 has an hardware failure
 * Partition transition to offline
 * This leader is replaced with a new broker with an empty disk and the same 
broker id 0
 * Partition transition from offline to online with leader 0, ISR = 0
 * Followers see the leader offset is 0 and decide to truncate their partitions 
to 0, ISR=0,1,2
 * At this point all the topic data has been removed from all replicas and 
partition size drops to 0 on all replicas

Attached some of the relevant logs. I can provide more logs if necessary

  was:
Sample setup:
 * a topic with one partition and RF=3
 * a producer using acks=1
 * min.insync.replicas to 1
 * 3 brokers 1,2,3
 * Preferred leader of the partition is brokerId 0

 

Steps to reproduce the issue
 * Producer keeps producing to the partition, leader is brokerId=0
 * At some point, replicas 1 and 2 are falling behind and removed from the ISR
 * The leader broker 0 has an hardware failure
 * Partition transition to offline
 * This leader is replaced with a new broker with an empty disk and the same 
broker id 0
 * Partition transition from offline to online with leader 0, ISR = 0
 * Followers see the leader offset is 0 and decide to truncate their partitions 
to 0, ISR=0,1,2

Attached some of the relevant logs. I can provide more logs if necessary


> Partitions are truncated when leader is replaced
> 
>
> Key: KAFKA-13872
> URL: https://issues.apache.org/jira/browse/KAFKA-13872
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.7.2
>Reporter: Francois Visconte
>Priority: Major
> Attachments: extract-2022-05-04T15_50_34.110Z.csv
>
>
> Sample setup:
>  * a topic with one partition and RF=3
>  * a producer using acks=1
>  * min.insync.replicas to 1
>  * 3 brokers 1,2,3
>  * Preferred leader of the partition is brokerId 0
>  
> Steps to reproduce the issue
>  * Producer keeps producing to the partition, leader is brokerId=0
>  * At some point, replicas 1 and 2 are falling behind and removed from the ISR
>  * The leader broker 0 has an hardware failure
>  * Partition transition to offline
>  * This leader is replaced with a new broker with an empty disk and the same 
> broker id 0
>  * Partition transition from offline to online with leader 0, ISR = 0
>  * Followers see the leader offset is 0 and decide to truncate their 
> partitions to 0, ISR=0,1,2
>  * At this point all the topic data has been removed from all replicas and 
> partition size drops to 0 on all replicas
> Attached some of the relevant logs. I can provide more logs if necessary



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Created] (KAFKA-13872) Partitions are truncated when leader is replaced

2022-05-04 Thread Francois Visconte (Jira)
Francois Visconte created KAFKA-13872:
-

 Summary: Partitions are truncated when leader is replaced
 Key: KAFKA-13872
 URL: https://issues.apache.org/jira/browse/KAFKA-13872
 Project: Kafka
  Issue Type: Bug
Affects Versions: 2.7.2
Reporter: Francois Visconte
 Attachments: extract-2022-05-04T15_50_34.110Z.csv

Sample setup:
 * a topic with one partition and RF=3
 * a producer using acks=1
 * min.insync.replicas to 1
 * 3 brokers 1,2,3
 * Preferred leader of the partition is brokerId 0

 

Steps to reproduce the issue
 * Producer keeps producing to the partition, leader is brokerId=0
 * At some point, replicas 1 and 2 are falling behind and removed from the ISR
 * The leader broker 0 has an hardware failure
 * Partition transition to offline
 * This leader is replaced with a new broker with an empty disk and the same 
broker id 0
 * Partition transition from offline to online with leader 0, ISR = 0
 * Followers see the leader offset is 0 and decide to truncate their partitions 
to 0, ISR=0,1,2

Attached some of the relevant logs. I can provide more logs if necessary



--
This message was sent by Atlassian Jira
(v8.20.7#820007)