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

Bingkun commented on KAFKA-17020:
---------------------------------

Hi [~jianbin] , thank you for your response. Actually we encountered the issue 
yesterday and the segment was successfully uploaded to S3. It's just the local 
segment doesn't get deleted. Also every time it happened, I checked the logs 
and it doesn't have any errors, just no doesn't have the delete operations. 
In your case, removing the setting log.roll.ms will make it to use the default 
value, which is 7 days. I'm wondering why make the default 7 days for for 
rolling can resolves the issue. In your case, did the log failed to roll when 
the issue happened?

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