[ https://issues.apache.org/jira/browse/KAFKA-17020?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17859873#comment-17859873 ]
Luke Chen commented on KAFKA-17020: ----------------------------------- I tried to reproduce the inconsistent log segment start offset, by creating 2 brokers setting log.roll.ms as different value. But both of them can delete logs as expected. {code:java} [2024-06-25 17:21:13,395] INFO [LocalLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs] Rolled new log segment at offset 6071460 in 2 ms. (kafka.log.LocalLog) [2024-06-25 17:21:13,395] INFO [ProducerStateManager partition=tieredTopic3-0] Wrote producer snapshot at offset 6071460 with 4 producer ids in 0 ms. (org.apache.kafka.storage.internals.log.ProducerStateManager) [2024-06-25 17:21:13,858] INFO [RemoteLogManager=2 partition=VHrBsdk9RmuyOR-6K8Qmdw:tieredTopic3-0] Copying 00000000000005036100.log to remote storage. (kafka.log.remote.RemoteLogManager$RLMTask) [2024-06-25 17:21:13,859] INFO Wait until the consumer is caught up with the target partition 0 up-to offset 40 (org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager) [2024-06-25 17:21:13,910] INFO [LocalTieredStorage Id=2] Offloading log segment for VHrBsdk9RmuyOR-6K8Qmdw:tieredTopic3-0 from segment=/tmp/kraft-broker-logs/tieredTopic3-0/00000000000005036100.log (org.apache.kafka.server.log.remote.storage.LocalTieredStorage) [2024-06-25 17:21:14,151] INFO [UnifiedLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs] Incremented local log start offset to 5036100 due to reason segment deletion (kafka.log.UnifiedLog) [2024-06-25 17:21:14,152] INFO [UnifiedLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs] Deleting segment LogSegment(baseOffset=4000740, size=1073737344, lastModifiedTime=1719307264358, largestRecordTimestamp=1719307264083) due to local log retention time 10000ms breach based on the largest record timestamp in the segment (kafka.log.UnifiedLog) [2024-06-25 17:21:14,317] INFO Wait until the consumer is caught up with the target partition 0 up-to offset 41 (org.apache.kafka.server.log.remote.metadata.storage.ConsumerManager) [2024-06-25 17:21:14,368] INFO [RemoteLogManager=2 partition=VHrBsdk9RmuyOR-6K8Qmdw:tieredTopic3-0] Copied 00000000000005036100.log to remote storage with segment-id: RemoteLogSegmentId{topicIdPartition=VHrBsdk9RmuyOR-6K8Qmdw:tieredTopic3-0, id=ySWB461eQ-2kNWI8DOqW6A} (kafka.log.remote.RemoteLogManager$RLMTask) [2024-06-25 17:21:15,154] INFO [LocalLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs] Deleting segment files LogSegment(baseOffset=4000740, size=1073737344, lastModifiedTime=1719307264358, largestRecordTimestamp=1719307264083) (kafka.log.LocalLog$) [2024-06-25 17:21:15,154] INFO Deleted producer state snapshot /tmp/kraft-broker-logs/tieredTopic3-0/00000000000004000740.snapshot.deleted (org.apache.kafka.storage.internals.log.SnapshotFile) [2024-06-25 17:21:15,182] INFO Deleted log /tmp/kraft-broker-logs/tieredTopic3-0/00000000000004000740.log.deleted. (org.apache.kafka.storage.internals.log.LogSegment) [2024-06-25 17:21:15,183] INFO Deleted offset index /tmp/kraft-broker-logs/tieredTopic3-0/00000000000004000740.index.deleted. (org.apache.kafka.storage.internals.log.LogSegment) [2024-06-25 17:21:15,183] INFO Deleted time index /tmp/kraft-broker-logs/tieredTopic3-0/00000000000004000740.timeindex.deleted. (org.apache.kafka.storage.internals.log.LogSegment) {code} {code:java} [2024-06-25 17:21:12,448] INFO [LocalLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Rolled new log segment at offset 5965125 in 2 ms. (kafka.log.LocalLog) [2024-06-25 17:21:12,448] INFO [ProducerStateManager partition=tieredTopic3-0] Wrote producer snapshot at offset 5965125 with 4 producer ids in 0 ms. (org.apache.kafka.storage.internals.log.ProducerStateManager) [2024-06-25 17:21:15,489] INFO [UnifiedLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Incremented local log start offset to 4935135 due to reason segment deletion (kafka.log.UnifiedLog) [2024-06-25 17:21:15,489] INFO [UnifiedLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Deleting segment LogSegment(baseOffset=4000740, size=969029908, lastModifiedTime=1719307263449, largestRecordTimestamp=1719307263197) due to local log retention time 10000ms breach based on the largest record timestamp in the segment (kafka.log.UnifiedLog) [2024-06-25 17:21:16,493] INFO [LocalLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Deleting segment files LogSegment(baseOffset=4000740, size=969029908, lastModifiedTime=1719307263449, largestRecordTimestamp=1719307263197) (kafka.log.LocalLog$) [2024-06-25 17:21:16,494] INFO Deleted producer state snapshot /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004000740.snapshot.deleted (org.apache.kafka.storage.internals.log.SnapshotFile) [2024-06-25 17:21:16,495] INFO Deleted log /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004000740.log.deleted. (org.apache.kafka.storage.internals.log.LogSegment) [2024-06-25 17:21:16,496] INFO Deleted offset index /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004000740.index.deleted. (org.apache.kafka.storage.internals.log.LogSegment) [2024-06-25 17:21:16,497] INFO Deleted time index /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004000740.timeindex.deleted. (org.apache.kafka.storage.internals.log.LogSegment) [2024-06-25 17:21:21,471] INFO [LocalLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Rolled new log segment at offset 6998520 in 1 ms. (kafka.log.LocalLog) [2024-06-25 17:21:21,472] INFO [ProducerStateManager partition=tieredTopic3-0] Wrote producer snapshot at offset 6998520 with 4 producer ids in 0 ms. (org.apache.kafka.storage.internals.log.ProducerStateManager) [2024-06-25 17:21:22,489] INFO [UnifiedLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Incremented local log start offset to 5965125 due to reason segment deletion (kafka.log.UnifiedLog) [2024-06-25 17:21:22,489] INFO [UnifiedLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Deleting segment LogSegment(baseOffset=4935135, size=1068168310, lastModifiedTime=1719307272447, largestRecordTimestamp=1719307272196) due to local log retention time 10000ms breach based on the largest record timestamp in the segment (kafka.log.UnifiedLog) [2024-06-25 17:21:23,489] INFO [LocalLog partition=tieredTopic3-0, dir=/tmp/kraft-broker-logs6] Deleting segment files LogSegment(baseOffset=4935135, size=1068168310, lastModifiedTime=1719307272447, largestRecordTimestamp=1719307272196) (kafka.log.LocalLog$) [2024-06-25 17:21:23,489] INFO Deleted producer state snapshot /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004935135.snapshot.deleted (org.apache.kafka.storage.internals.log.SnapshotFile) [2024-06-25 17:21:23,587] INFO Deleted log /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004935135.log.deleted. (org.apache.kafka.storage.internals.log.LogSegment) [2024-06-25 17:21:23,587] INFO Deleted offset index /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004935135.index.deleted. (org.apache.kafka.storage.internals.log.LogSegment) [2024-06-25 17:21:23,588] INFO Deleted time index /tmp/kraft-broker-logs6/tieredTopic3-0/00000000000004935135.timeindex.deleted. (org.apache.kafka.storage.internals.log.LogSegment) {code} > 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)