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

Jianbin Chen commented on KAFKA-17020:
--------------------------------------

I dumped the logs and it appears the leader only has two log segments. One of 
them, 00000000000032117950, was uploaded to remote storage and then deleted; 
the replica shows the same behavior. The strange thing is the replica rolled a 
new segment — 00000000000032119412 — which is 512 MB, but there are no 
corresponding logs on the leader about this segment, so I don't know where it 
came from.

Checking remote storage, I found segment 00000000000032117950 present, but 
00000000000032119412 does not exist. In other words, there are actually only 
two segments. I don't understand why the replica ended up with an extra 
segment, and that explains why the extra segment isn't being deleted: the 
leader simply doesn't have a matching third segment.

!image-2025-09-11-10-31-40-896.png|width=645,height=162!

replica:

 
{code:java}
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:50:38,223] INFO [UnifiedLog 
partition=kunpeng_api_raw_activity_efficient-15, 
dir=/home/admin/output/kafka-logs] Incremented local log start offset to 
32119412 due to reason segment deletion (kafka.log.UnifiedLog) 
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:50:38,223] INFO [UnifiedLog 
partition=kunpeng_api_raw_activity_efficient-15, 
dir=/home/admin/output/kafka-logs] Deleting segment 
LogSegment(baseOffset=32117950, size=10477860, lastModifiedTime=1757496641724, 
largestRecordTimestamp=1757488182176) due to local log retention time 600000ms 
breach based on the largest record timestamp in the segment 
(kafka.log.UnifiedLog)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:51:38,223] INFO [LocalLog 
partition=kunpeng_api_raw_activity_efficient-15, 
dir=/home/admin/output/kafka-logs] Deleting segment files 
LogSegment(baseOffset=32117950, size=10477860, lastModifiedTime=1757496641724, 
largestRecordTimestamp=1757488182176) (kafka.log.LocalLog$)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:51:38,240] INFO Deleted log 
/home/admin/output/kafka-logs/kunpeng_api_raw_activity_efficient-15/00000000000032117950.log.deleted.
 (org.apache.kafka.storage.internals.log.LogSegment) 
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:51:38,240] INFO Deleted 
producer state snapshot 
/home/admin/output/kafka-logs/kunpeng_api_raw_activity_efficient-15/00000000000032117950.snapshot.deleted
 (org.apache.kafka.storage.internals.log.SnapshotFile)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:51:38,240] INFO Deleted 
offset index 
/home/admin/output/kafka-logs/kunpeng_api_raw_activity_efficient-15/00000000000032117950.index.deleted.
 (org.apache.kafka.storage.internals.log.LogSegment)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:51:38,240] INFO Deleted time 
index 
/home/admin/output/kafka-logs/kunpeng_api_raw_activity_efficient-15/00000000000032117950.timeindex.deleted.
 (org.apache.kafka.storage.internals.log.LogSegment)
kafka/logs/server.log.2025-09-11-03:[2025-09-11 03:17:59,072] INFO [LocalLog 
partition=kunpeng_api_raw_activity_efficient-15, 
dir=/home/admin/output/kafka-logs] Rolled new log segment at offset 32195041 in 
0 ms. (kafka.log.LocalLog) 
kafka/logs/server.log.2025-09-11-03:[2025-09-11 03:17:59,074] INFO 
[ProducerStateManager partition=kunpeng_api_raw_activity_efficient-15] Wrote 
producer snapshot at offset 32195041 with 0 producer ids in 0 ms. 
(org.apache.kafka.storage.internals.log.ProducerStateManager)
{code}
leader: 
{code:java}
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:47:46,336] INFO [LocalLog 
partition=kunpeng_api_raw_activity_efficient-15, dir=/home/admin/output/
kafka-logs] Rolled new log segment at offset 32193569 in 2 ms. 
(kafka.log.LocalLog)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:47:46,337] INFO 
[ProducerStateManager partition=kunpeng_api_raw_activity_efficient-15] Wrote 
produ
cer snapshot at offset 32193569 with 0 producer ids in 0 ms. 
(org.apache.kafka.storage.internals.log.ProducerStateManager)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:48:11,567] INFO 
[RemoteLogManager=3 
partition=MjWTqzHUSzGAMUM1F02JbA:kunpeng_api_raw_activity_effi
cient-15] Copying 00000000000032117950.log to remote storage. 
(kafka.log.remote.RemoteLogManager$RLMCopyTask)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:48:11,577] INFO Copying log 
segment data, metadata: RemoteLogSegmentMetadata{remoteLogSegmentId=Re
moteLogSegmentId{topicIdPartition=MjWTqzHUSzGAMUM1F02JbA:kunpeng_api_raw_activity_efficient-15,
 id=ma0wUF-JSJOjX35Q89rDCQ}, startOffset=32117950, end
Offset=32193568, brokerId=3, maxTimestampMs=1757530065562, 
eventTimestampMs=1757530091567, segmentLeaderEpochs={14=32117950, 15=32118572, 
17=32128055, 18=32128540, 19=32132704, 20=32132768}, 
segmentSizeInBytes=536868812, customMetadata=Optional.empty, 
state=COPY_SEGMENT_STARTED} (io.aiven.kafka.tieredstorage.RemoteStorageManager)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:48:22,397] INFO Copying log 
segment data completed successfully, metadata: 
RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=MjWTqzHUSzGAMUM1F02JbA:kunpeng_api_raw_activity_efficient-15,
 id=ma0wUF-JSJOjX35Q89rDCQ}, startOffset=32117950, endOffset=32193568, 
brokerId=3, maxTimestampMs=1757530065562, eventTimestampMs=1757530091567, 
segmentLeaderEpochs={14=32117950, 15=32118572, 17=32128055, 18=32128540, 
19=32132704, 20=32132768}, segmentSizeInBytes=536868812, 
customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED} 
(io.aiven.kafka.tieredstorage.RemoteStorageManager)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:48:22,452] INFO 
[RemoteLogManager=3 
partition=MjWTqzHUSzGAMUM1F02JbA:kunpeng_api_raw_activity_efficient-15] Copied 
00000000000032117950.log to remote storage with segment-id: 
RemoteLogSegmentId{topicIdPartition=MjWTqzHUSzGAMUM1F02JbA:kunpeng_api_raw_activity_efficient-15,
 id=ma0wUF-JSJOjX35Q89rDCQ} (kafka.log.remote.RemoteLogManager$RLMCopyTask)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:58:20,929] INFO [UnifiedLog 
partition=kunpeng_api_raw_activity_efficient-15, 
dir=/home/admin/output/kafka-logs] Incremented local log start offset to 
32193569 due to reason segment deletion (kafka.log.UnifiedLog)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:58:20,931] INFO [UnifiedLog 
partition=kunpeng_api_raw_activity_efficient-15, 
dir=/home/admin/output/kafka-logs] Deleting segment 
LogSegment(baseOffset=32117950, size=536868812, lastModifiedTime=1757530065993, 
largestRecordTimestamp=1757530065562) due to local log retention time 600000ms 
breach based on the largest record timestamp in the segment 
(kafka.log.UnifiedLog)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:59:20,932] INFO [LocalLog 
partition=kunpeng_api_raw_activity_efficient-15, 
dir=/home/admin/output/kafka-logs] Deleting segment files 
LogSegment(baseOffset=32117950, size=536868812, lastModifiedTime=1757530065993, 
largestRecordTimestamp=17575300655
62) (kafka.log.LocalLog$)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:59:20,933] INFO Deleted 
producer state snapshot /home/admin/output/kafka-logs/kunpeng_api_raw_acti
vity_efficient-15/00000000000032117950.snapshot.deleted 
(org.apache.kafka.storage.internals.log.SnapshotFile)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:59:20,939] INFO Deleted log 
/home/admin/output/kafka-logs/kunpeng_api_raw_activity_efficient-15/00
000000000032117950.log.deleted. 
(org.apache.kafka.storage.internals.log.LogSegment)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:59:20,939] INFO Deleted 
offset index /home/admin/output/kafka-logs/kunpeng_api_raw_activity_effici
ent-15/00000000000032117950.index.deleted. 
(org.apache.kafka.storage.internals.log.LogSegment)
kafka/logs/server.log.2025-09-11-02:[2025-09-11 02:59:20,940] INFO Deleted time 
index /home/admin/output/kafka-logs/kunpeng_api_raw_activity_efficien
t-15/00000000000032117950.timeindex.deleted. 
(org.apache.kafka.storage.internals.log.LogSegment) {code}
[~showuon]  PTAL

 

> After enabling tiered storage, occasional residual logs are left in the 
> replica
> -------------------------------------------------------------------------------
>
>                 Key: KAFKA-17020
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17020
>             Project: Kafka
>          Issue Type: Wish
>    Affects Versions: 3.7.0, 3.9.1
>            Reporter: Jianbin Chen
>            Priority: Major
>         Attachments: image-2024-06-22-21-45-43-815.png, 
> image-2024-06-22-21-46-12-371.png, image-2024-06-22-21-46-26-530.png, 
> image-2024-06-22-21-46-42-917.png, image-2024-06-22-21-47-00-230.png, 
> image-2025-09-11-09-57-18-234.png, image-2025-09-11-09-57-57-746.png, 
> image-2025-09-11-10-31-40-896.png
>
>
> After enabling tiered storage, occasional residual logs are left in the 
> replica.
> Based on the observed phenomenon, the index values of the rolled-out logs 
> generated by the replica and the leader are not the same. As a result, the 
> logs uploaded to S3 at the same time do not include the corresponding log 
> files on the replica side, making it impossible to delete the local logs.
> !image-2024-06-22-21-45-43-815.png!
> leader config:
> {code:java}
> num.partitions=3
> default.replication.factor=2
> delete.topic.enable=true
> auto.create.topics.enable=false
> num.recovery.threads.per.data.dir=1
> offsets.topic.replication.factor=3
> transaction.state.log.replication.factor=2
> transaction.state.log.min.isr=1
> offsets.retention.minutes=4320
> log.roll.ms=86400000
> log.local.retention.ms=600000
> log.segment.bytes=536870912
> num.replica.fetchers=1
> log.retention.ms=15811200000
> remote.log.manager.thread.pool.size=4
> remote.log.reader.threads=4
> remote.log.metadata.topic.replication.factor=3
> remote.log.storage.system.enable=true
> remote.log.metadata.topic.retention.ms=180000000
> rsm.config.fetch.chunk.cache.class=io.aiven.kafka.tieredstorage.fetch.cache.DiskChunkCache
> rsm.config.fetch.chunk.cache.path=/data01/kafka-tiered-storage-cache
> Pick some cache size, 16 GiB here:
> rsm.config.fetch.chunk.cache.size=34359738368
> rsm.config.fetch.chunk.cache.retention.ms=1200000
> # # Prefetching size, 16 MiB here:
> rsm.config.fetch.chunk.cache.prefetch.max.size=33554432
> rsm.config.storage.backend.class=io.aiven.kafka.tieredstorage.storage.s3.S3Storage
> rsm.config.storage.s3.bucket.name=
> rsm.config.storage.s3.region=us-west-1
> rsm.config.storage.aws.secret.access.key=
> rsm.config.storage.aws.access.key.id=
> rsm.config.chunk.size=8388608
> remote.log.storage.manager.class.path=/home/admin/core-0.0.1-SNAPSHOT/:/home/admin/s3-0.0.1-SNAPSHOT/
> 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=PLAINTEXT
> rsm.config.upload.rate.limit.bytes.per.second=31457280
> {code}
>  replica config:
> {code:java}
> num.partitions=3
> default.replication.factor=2
> delete.topic.enable=true
> auto.create.topics.enable=false
> num.recovery.threads.per.data.dir=1
> offsets.topic.replication.factor=3
> transaction.state.log.replication.factor=2
> transaction.state.log.min.isr=1
> offsets.retention.minutes=4320
> log.roll.ms=86400000
> log.local.retention.ms=600000
> log.segment.bytes=536870912
> num.replica.fetchers=1
> log.retention.ms=15811200000
> remote.log.manager.thread.pool.size=4
> remote.log.reader.threads=4
> remote.log.metadata.topic.replication.factor=3
> remote.log.storage.system.enable=true
> #remote.log.metadata.topic.retention.ms=180000000
> rsm.config.fetch.chunk.cache.class=io.aiven.kafka.tieredstorage.fetch.cache.DiskChunkCache
> rsm.config.fetch.chunk.cache.path=/data01/kafka-tiered-storage-cache
> # Pick some cache size, 16 GiB here:
> rsm.config.fetch.chunk.cache.size=34359738368
> rsm.config.fetch.chunk.cache.retention.ms=1200000
> # # # Prefetching size, 16 MiB here:
> rsm.config.fetch.chunk.cache.prefetch.max.size=33554432
> rsm.config.storage.backend.class=io.aiven.kafka.tieredstorage.storage.s3.S3Storage
> rsm.config.storage.s3.bucket.name=
> rsm.config.storage.s3.region=us-west-1
> rsm.config.storage.aws.secret.access.key=
> rsm.config.storage.aws.access.key.id=
> rsm.config.chunk.size=8388608
> remote.log.storage.manager.class.path=/home/admin/core-0.0.1-SNAPSHOT/*:/home/admin/s3-0.0.1-SNAPSHOT/*
> 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=PLAINTEXT
> rsm.config.upload.rate.limit.bytes.per.second=31457280 {code}
> topic config:
> {code:java}
> Dynamic configs for topic xxxxxx are:
> local.retention.ms=600000 sensitive=false 
> synonyms={DYNAMIC_TOPIC_CONFIG:local.retention.ms=600000, 
> STATIC_BROKER_CONFIG:log.local.retention.ms=600000, 
> DEFAULT_CONFIG:log.local.retention.ms=-2}
> remote.storage.enable=true sensitive=false 
> synonyms={DYNAMIC_TOPIC_CONFIG:remote.storage.enable=true}
> retention.ms=15811200000 sensitive=false 
> synonyms={DYNAMIC_TOPIC_CONFIG:retention.ms=15811200000, 
> STATIC_BROKER_CONFIG:log.retention.ms=15811200000, 
> DEFAULT_CONFIG:log.retention.hours=168}
> segment.bytes=536870912 sensitive=false 
> synonyms={DYNAMIC_TOPIC_CONFIG:segment.bytes=536870912, 
> STATIC_BROKER_CONFIG:log.segment.bytes=536870912, 
> DEFAULT_CONFIG:log.segment.bytes=1073741824} {code}
>  
> !image-2024-06-22-21-46-12-371.png!
> By examining the segment logs for that time period in S3 for the topic, it 
> can be observed that the indices of the two are different.
> !image-2024-06-22-21-46-26-530.png!
> By searching for the residual log index through log analysis, it was found 
> that there were no delete logs on both the leader and replica nodes. However, 
> the logs for the corresponding time period in S3 can be queried in the leader 
> node logs but not in the replica node logs. Therefore, I believe that the 
> issue is due to the different log files generated by the leader and replica 
> nodes.
> !image-2024-06-22-21-46-42-917.png!
> {color:#172b4d}Restarting does not resolve this issue. The only solution is 
> to delete the log folder corresponding to the replica where the log segment 
> anomaly occurred and then resynchronize from the leader.{color}
> !image-2024-06-22-21-47-00-230.png!



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

Reply via email to