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

Arpit Goyal commented on KAFKA-15388:
-------------------------------------

Hey [~divijvaidya] I am successfully able to reproduce the issue locally. 
*Configuration* 
1. Create Topic name - test20 partition 0 
2. add segment.bytes as 100 for quicker roll of segments 
{code:java}
bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type 
topics --entity-name test20  --add-config segment.bytes=100
{code}
3. Enable clean up compact policy 
{code:java}
bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type 
topics --entity-name test20  --add-config cleanup.policy=compact
{code} 
4. *Produce some messages *

{code:java}
bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test20 
--property "parse.key=true" --property "key.separator=:"
>1:1
>1:2
>1:3
>1:4
>1:5
>1:6
>1:7
>1:8
>1:9
>1:10
>1:11
>1:12
>1:13
>1:14
>1:15
>1:16
{code}
5. When we try to consume message from the topic without remote storage enable 
{code:java}
(base) ➜  kafka git:(trunk) ✗ bin/kafka-console-consumer.sh  --bootstrap-server 
localhost:9092 --topic test20  --offset 0 --partition 0 --property 
print.offset=true
Offset:14       15
Offset:15       16
{code}
6. Disable the compact policy 
{code:java}
bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type 
topics --entity-name test20  --delete-config cleanup.policy
{code}
7. Enable remote storage for the required topic 
{code:java}
bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --entity-type 
topics --entity-name test20  --add-config remote.storage.enable=true
{code}
8. The rolled over segments has been moved to tiered storage topic. Attached 
screenshot for the reference. (tieredtopicconfiglist)
9. As it was a historically compacted topic with only key 1 , most of the log 
segments had zero bytes.Please refer (tieredtopicconfiglist) screenshot.Only 
14.log and 15.log has some data. 
10. When we try fetching it from the remote storage , it looks for the first 
batch assuming it would always exist , but in this case batch always return 
null because segment is empty and batch would always be null (  0 >= 0 -11)
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java#L65
https://github.com/apache/kafka/blob/trunk/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L1411
https://github.com/apache/kafka/blob/trunk/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L1277

11. This seems to be a critical bug , and it should be inline with the local 
log fetch mechanism , ie if segment is null , it should fetch the higher 
segment and continue till the end offset. 

Should i create a different bug for this ? 







> Handle topics that were having compaction as retention earlier are changed to 
> delete only retention policy and onboarded to tiered storage. 
> --------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-15388
>                 URL: https://issues.apache.org/jira/browse/KAFKA-15388
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Satish Duggana
>            Assignee: Arpit Goyal
>            Priority: Blocker
>             Fix For: 3.7.0
>
>         Attachments: Screenshot 2023-11-15 at 3.47.54 PM.png, 
> tieredtopicloglist.png
>
>
> Context: [https://github.com/apache/kafka/pull/13561#discussion_r1300055517]
>  
> There are 3 paths I looked at:
>  * When data is moved to remote storage (1)
>  * When data is read from remote storage (2)
>  * When data is deleted from remote storage (3)
> (1) Does not have a problem with compacted topics. Compacted segments are 
> uploaded and their metadata claims they contain offset from the baseOffset of 
> the segment until the next segment's baseOffset. There are no gaps in offsets.
> (2) Does not have a problem if a customer is querying offsets which do not 
> exist within a segment, but there are offset after the queried offset within 
> the same segment. *However, it does have a problem when the next available 
> offset is in a subsequent segment.*
> (3) For data deleted via DeleteRecords there is no problem. For data deleted 
> via retention there is no problem.
>  
> *I believe the proper solution to (2) is to make tiered storage continue 
> looking for the next greater offset in subsequent segments.*
> Steps to reproduce the issue:
> {code:java}
> // TODO (christo)
> {code}



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

Reply via email to