aiven / kafka

Mirror of Apache Kafka
Apache License 2.0
2 stars 1 forks source link

Broker may forget to upload segment to remote storage - data loss #18

Closed ivanyu closed 1 year ago

ivanyu commented 1 year ago

Start a broker with the following configuration:

remote.log.storage.system.enable=true
remote.log.manager.task.interval.ms=5000
remote.log.metadata.manager.class.name=org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager
rlmm.config.remote.log.metadata.topic.replication.factor=1
remote.log.metadata.manager.listener.name=PLAINTEXT
remote.log.storage.manager.class.name=io.aiven.kafka.tieredstorage.commons.UniversalRemoteStorageManager
rsm.config.chunk.size=102400
rsm.config.object.storage.factory=io.aiven.kafka.tieredstorage.commons.storage.filesystem.FileSystemStorageFactory
rsm.config.object.storage.root=<some dir here>

(it uses our UniversalRemoteStorageManager, but it doesn't matter).

Create the topic with the following command:

~/kafka/kafka_2.13-3.4.0/bin/kafka-topics.sh --bootstrap-server localhost:9092 \
    --create --topic topic1 \
    --config remote.storage.enable=true \
    --config segment.bytes=512000 \
    --config retention.ms=3600000 \
    --config local.retention.ms=120000

Fill it with the following command:

~/kafka/kafka_2.13-3.4.0/bin/kafka-producer-perf-test.sh \
    --topic topic1 --num-records=10000 --throughput -1 --record-size 1000 \
    --producer-props acks=1 batch.size=16384 bootstrap.servers=localhost:9092

The last/active segment is 00000000000000009920.

Now happens this:

  1. All the current segments on the disk are uploaded (save for the last, i.e. active).

    [2023-04-21 07:48:16,094] INFO [RemoteLogManager=0 partition=o46XK3njSDKNWM2AuTeVuQ:topic1-0] Copied 00000000000000000000.log to remote storage with segment-id: RemoteLogSegmentId{topicIdPartition=o46XK3njSDKNWM2AuTeVuQ:topic1-0, id=xw7n8ttqSZKdaZFMDveVvg} (kafka.log.remote.RemoteLogManager$RLMTask)
    ...
    [2023-04-21 07:48:18,218] INFO [RemoteLogManager=0 partition=o46XK3njSDKNWM2AuTeVuQ:topic1-0] Copied 00000000000000009424.log to remote storage with segment-id: RemoteLogSegmentId{topicIdPartition=o46XK3njSDKNWM2AuTeVuQ:topic1-0, id=kAjnjNskSAOn2x4xnGQqlA} (kafka.log.remote.RemoteLogManager$RLMTask)
  2. A new segment is rolled (don't know why, the data aren't being written, but it doesn't matter)

    [2023-04-21 07:50:32,317] INFO [LocalLog partition=topic1-0, dir=/home/ivanyu/kafka/kafka-ts-development/_run-kafka-logs] Rolled new log segment at offset 10000 in 5 ms. (kafka.log.LocalLog)
    [2023-04-21 07:50:32,335] INFO [ProducerStateManager partition=topic1-0] Wrote producer snapshot at offset 10000 with 0 producer ids in 18 ms. (kafka.log.ProducerStateManager)
  3. The 00000000000000009920 becomes eligible for uploading and cleaning, but it is not uploaded.

  4. Old segments are cleaned

    [2023-04-21 07:51:32,342] INFO Deleted log /home/ivanyu/kafka/kafka-ts-development/_run-kafka-logs/topic1-0/00000000000000000000.log.deleted. (kafka.log.LogSegment)
    ...
    [2023-04-21 07:51:32,352] INFO Deleted log /home/ivanyu/kafka/kafka-ts-development/_run-kafka-logs/topic1-0/00000000000000009920.log.deleted. (kafka.log.LogSegment)

    including the 00000000000000009920, which is not uploaded.

mdedetrich commented 1 year ago

On which branch is this happening?

ivanyu commented 1 year ago

3.3-2022-10-06-tiered-storage

mdedetrich commented 1 year ago

Did you try running on 3.0 to see if its also there?

ivanyu commented 1 year ago

Reproduces on 3.3-2022-10-06-tiered-storage, but works as expected on 3.0-li. The 10000 segment is not rolled on 3.0, so the 9920 remains active and not deleted.

ivanyu commented 1 year ago

It seems, on 3.3 it reproduces inconsistently, so I can't guarantee it doesn't happen on 3.0 as well.

ivanyu commented 1 year ago

UPD: no, it's consistent. The segment roll happens as the same moment as first deletion:

[2023-04-21 13:33:12,765] INFO [LocalLog partition=topic1-0, dir=/home/ivanyu/kafka/kafka-ts-development/_run-kafka-logs] Rolled new log segment at offset 10000 in 4 ms. (kafka.log.LocalLog)
[2023-04-21 13:33:12,784] INFO [ProducerStateManager partition=topic1-0] Wrote producer snapshot at offset 10000 with 0 producer ids in 19 ms. (kafka.log.ProducerStateManager)
[2023-04-21 13:33:12,786] INFO [UnifiedLog partition=topic1-0, dir=/home/ivanyu/kafka/kafka-ts-development/_run-kafka-logs] Deleting segment LogSegment(baseOffset=0, size=502355, lastModifiedTime=1682073046965, largestRecordTimestamp=Some(1682073046848)) due to retention time 120000ms breach based on the largest record timestamp in the segment (kafka.log.UnifiedLog)
mdedetrich commented 1 year ago

@ivanyu I believe I have found the root cause for the fix, can you apply this patch onto the 3.3 branch and confirm whether it solves the problem or not?

Fix_for_accidental_deletion_of_remote_log_segments.patch

ivanyu commented 1 year ago

Seems to be working with the patch: the 10000 segment is not created and the 9920 one remains the active.

mdedetrich commented 1 year ago

Perfect thanks! I need to update my github subscriptions, I missed this.

Force updating the branch now with the patch.

mdedetrich commented 1 year ago

Closing ticket, the 3.3-2022-10-06-tiered-storage has force pushed with the fix.