aiven / kafka

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

Inconsistent segments archiving/deleting for size-based retention in 3.3 #34

Closed jeqo closed 1 month ago

jeqo commented 1 year ago

Testing archival and deleting find out that segments expected to be deleted are not, or segments are archived when they shouldn’t

How to reproduce

Testing with https://github.com/aiven/tiered-storage-for-apache-kafka/tree/main/demo

Happy path (no TS)

First, create a topic without TS:

docker exec kafka-ts \
        kafka-topics \
        --bootstrap-server kafka:29092 \
        --create \
        --config segment.bytes=1000000  \
        --config retention.bytes=20000000  \
        --partitions 1 \
        --topic t1-nots
Created topic t1-nots.

20MB retention, 1MB segment files. Expected segments: 20-21 (including active)

Fill topic with data (~60MB):

docker exec kafka-ts \
        kafka-producer-perf-test --producer-props bootstrap.servers=kafka:29092 \
        --topic t1-nots \
        --num-records 60000 \
        --record-size 1024 \
        --throughput 1000

Check local segment files:

make topic=t1-nots show_local_data | grep '.log$' | wc -l

Should return 20 or 21 segments as expected.

And segments should include latest ~20K messages:

make topic=t1-nots show_local_data | grep '.log$'
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:35 00000000000000041217.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:35 00000000000000042151.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:35 00000000000000043086.log
-rw-r--r--. 1 appuser appuser 975K Jun 27 09:35 00000000000000044021.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:35 00000000000000044954.log
# ...
-rw-r--r--. 1 appuser appuser 977K Jun 27 09:35 00000000000000056153.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:35 00000000000000057090.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:35 00000000000000058024.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:35 00000000000000058960.log
-rw-r--r--. 1 appuser appuser 108K Jun 27 09:35 00000000000000059897.log

With TS enabled

Create topic with TS enabled, local retention of 5MB

docker exec kafka-ts \
        kafka-topics \
        --bootstrap-server kafka:29092 \
        --create \
        --config remote.storage.enable=true \
        --config segment.bytes=1000000  \
        --config retention.bytes=20000000  \
        --config local.retention.bytes=5000000  \
        --partitions 1 \
        --topic t1-ts
Created topic t1-ts.

It’s expected to have 5 or 6 segments locally, and at least 15 segments remotely.

Let’s fill the topic:

docker exec kafka-ts \
        kafka-producer-perf-test --producer-props bootstrap.servers=kafka:29092 \
        --topic t1-ts \
        --num-records 60000 \
        --record-size 1024 \
        --throughput 1000

Monitor remote segments:

make topic=t1-ts show_remote_data_s3_minio | grep '.log$' | wc -l

In the latest test, it returns 10, and local segments return 6

Checking the segments in more detail:

Local segments look ok:

-rw-r--r--. 1 appuser appuser 975K Jun 27 09:44 00000000000000055245.log
-rw-r--r--. 1 appuser appuser 975K Jun 27 09:44 00000000000000056179.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:44 00000000000000057115.log
-rw-r--r--. 1 appuser appuser 977K Jun 27 09:44 00000000000000058052.log
-rw-r--r--. 1 appuser appuser 976K Jun 27 09:44 00000000000000058991.log
-rw-r--r--. 1 appuser appuser  74K Jun 27 09:44 00000000000000059929.log

But remote segments seems suspicious with 10 segments:

[2023-06-27 09:44:34 UTC] 975KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000050565-YsYKhWHnR4G_QeHOwoMGUA.log
[2023-06-27 09:44:34 UTC] 976KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000051505-SEUI-ie8T2WH2_fSuQsmHA.log
[2023-06-27 09:44:34 UTC] 976KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000052441-YvjikTNCR9C3R08dhiZM0Q.log
[2023-06-27 09:44:35 UTC] 975KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000053376-uzqfbrt6QrmJucjAm-DFCg.log
[2023-06-27 09:44:35 UTC] 976KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000054308-Z1ZTbREMShW5yhF-RN_pzQ.log
[2023-06-27 09:44:35 UTC] 975KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000055245-EFECwnOSS2-y_Un_dlglSw.log
[2023-06-27 09:44:42 UTC] 975KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000056179-V4cfN4LCS9OATFU9gnshyA.log
[2023-06-27 09:44:43 UTC] 976KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000057115-MMFiR96dR4KsFUvYhlySeA.log
[2023-06-27 09:44:43 UTC] 976KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000058052-_mAKtQx3SMqARrpu7owLNw.log
[2023-06-27 09:44:43 UTC] 976KiB STANDARD tiered-storage-demo/t1-ts-Etgijt_mRCe-N8WW_Pac6Q/0/00000000000000058991-leZD6nzLSf62IN9C2XW8Aw.log

First of all, it contains less than expected (15), and it’s most recent segment overlaps with local segments (even after test stops) most recent remote log starts at 00000000000000058991 but oldest log starts at 00000000000000055245

Also, it does not contain the 15 expected segments leading to data lost compared to no TS: 11 segments are missing.

Potential causes

PS. Seems upstream current work on implementing retention is already handling this conditions: https://github.com/apache/kafka/pull/13561/files#diff-380e4d8859ea9148f21794c09039425c82d9012a392c2dbbe1ce2ec8677a1970R804