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

Chia-Ping Tsai commented on KAFKA-16310:
----------------------------------------

the bug is related to https://issues.apache.org/jira/browse/KAFKA-16310

the code of updating `offsetOfMaxTimestamp` was:

{code:scala}
    if (timestampType == TimestampType.LOG_APPEND_TIME) {
      maxTimestamp = now
      if (magic >= RecordBatch.MAGIC_VALUE_V2)
        offsetOfMaxTimestamp = offsetCounter.value - 1
      else
        offsetOfMaxTimestamp = initialOffset
    }
{code}

and now we always update `offsetOfMaxTimestamp` to `offsetCounter.value - 1` 
even though it is  TimestampType.LOG_APPEND_TIME (see 
https://github.com/apache/kafka/blob/trunk/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java#L300)
{code:java}
        if (timestampType == TimestampType.LOG_APPEND_TIME) {
            maxTimestamp = now;
            offsetOfMaxTimestamp = initialOffset;
        }

        if (toMagic >= RecordBatch.MAGIC_VALUE_V2) {
            offsetOfMaxTimestamp = offsetCounter.value - 1;
        }
{code}

Furthermore, the path of compressed data has similar issue. see 
https://github.com/apache/kafka/blob/trunk/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java#L417

[~kevinztw] [~ijuma] Please double check the root cause, thanks!


> ListOffsets doesn't report the offset with maxTimestamp anymore
> ---------------------------------------------------------------
>
>                 Key: KAFKA-16310
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16310
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 3.7.0
>            Reporter: Emanuele Sabellico
>            Assignee: Cheng-Kai, Zhang
>            Priority: Minor
>
> The last offset is reported instead.
> A test in librdkafka (0081/do_test_ListOffsets) is failing an it's checking 
> that the offset with the max timestamp is the middle one and not the last 
> one. The tests is passing with 3.6.0 and previous versions
> This is the test:
> [https://github.com/confluentinc/librdkafka/blob/a6d85bdbc1023b1a5477b8befe516242c3e182f6/tests/0081-admin.c#L4989]
>  
> there are three messages, with timestamps:
> {noformat}
> t0 + 100
> t0 + 400
> t0 + 250{noformat}
> and indices 0,1,2. 
> then a ListOffsets with RD_KAFKA_OFFSET_SPEC_MAX_TIMESTAMP is done.
> it should return offset 1 but in 3.7.0 and trunk is returning offset 2
> Even after 5 seconds from producing it's still returning 2 as the offset with 
> max timestamp.
> ProduceRequest and ListOffsets were sent to the same broker (2), the leader 
> didn't change.
> {code:java}
> %7|1709134230.019|SEND|0081_admin#producer-3| 
> [thrd:localhost:39951/bootstrap]: localhost:39951/2: Sent ProduceRequest (v7, 
> 206 bytes @ 0, CorrId 2) %7|1709134230.020|RECV|0081_admin#producer-3| 
> [thrd:localhost:39951/bootstrap]: localhost:39951/2: Received ProduceResponse 
> (v7, 95 bytes, CorrId 2, rtt 1.18ms) 
> %7|1709134230.020|MSGSET|0081_admin#producer-3| 
> [thrd:localhost:39951/bootstrap]: localhost:39951/2: 
> rdkafkatest_rnd22e8d8ec45b53f98_do_test_ListOffsets [0]: MessageSet with 3 
> message(s) (MsgId 0, BaseSeq -1) delivered {code}
> {code:java}
> %7|1709134235.021|SEND|0081_admin#producer-2| 
> [thrd:localhost:39951/bootstrap]: localhost:39951/2: Sent ListOffsetsRequest 
> (v7, 103 bytes @ 0, CorrId 7) %7|1709134235.022|RECV|0081_admin#producer-2| 
> [thrd:localhost:39951/bootstrap]: localhost:39951/2: Received 
> ListOffsetsResponse (v7, 88 bytes, CorrId 7, rtt 0.54ms){code}



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

Reply via email to