[ https://issues.apache.org/jira/browse/KAFKA-15388?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17783559#comment-17783559 ]
Divij Vaidya commented on KAFKA-15388: -------------------------------------- Hey [~goyarpit] - apologies for the delay. I will intermittently respond to this ticket in the next few weeks as I am going on a vacation. Let me try to answer your pending queries: > Last offset of record batch will always be equal to the value before > compaction. Correct. When compaction creates a new merged segment, it keeps the base and end offset same as previous non-compacted segments. Which means that segment base and last offset will always be continuous even if the data inside has been removed/compacted. > Now if we try to fetch data for 50th offset it would return null. Is it > something expected or a bug ? It's a bug. You can check how local storage handles it at [https://github.com/apache/kafka/blob/317f61dfd9a7f1443cf75b6a32568d1c81984d08/core/src/main/scala/kafka/log/LocalLog.scala#L425] We look into the next higher segment i.e. in this case we will look into RB3 and offsets starting from 51. Note that read() calls translateOffset() which returns the first message with offset >= the requested offset. Note the > here. That is how 51 will be returned. We need to ensure that RemoteLogManager.read() behaves the same way. I would suggest that you try the following and this should fail: 1. Create multiple segments 2. Compact away (the last offsets in a few of these segments) 3. disable compaction. Turn on Tiered Storage. 4. These segments should get uploaded successfully. 5. Try to read the last offset which was compacted away. As per the read contract of offsets which are removed during compaction, you should get available offset after that offset. i.e. if you are asking for 3 and 3 has been removed by compaction, you will get 4 assuming 4 is available. This should work for TS when we are reading 3 from TS. > 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 > > > 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)