[
https://issues.apache.org/jira/browse/KAFKA-16310?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17831658#comment-17831658
]
Chia-Ping Tsai commented on KAFKA-16310:
----------------------------------------
{quote}
I think the easiest way to fix the listMaxTimestamp issue is probably to still
maintain offsetOfMaxTimestamp at the record batch level so that it can be
derived consistently at both the leader and the follower. When serving the
listMaxTimestamp request, we iterate the batch containing the maxTimestamp to
find the exact record offset with maxTimestamp. Since this is a rare operation,
paying the decompression overhead is fine. What do you think?
{quote}
I'm still digging in :)
BTW, it seems the recovering the segments has similar issue.
{code}
// The max timestamp is exposed at the batch level, so no need
to iterate the records
if (batch.maxTimestamp() > maxTimestampSoFar()) {
maxTimestampAndOffsetSoFar = new
TimestampOffset(batch.maxTimestamp(), batch.lastOffset());
}
{code}
> 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: Chia-Ping Tsai
> Priority: Blocker
> Fix For: 3.8.0, 3.7.1
>
>
> Updated: This is confirmed a regression issue in v3.7.0.
> The impact of this issue is that when there is a batch containing records
> with timestamp not in order, the offset of the timestamp will be wrong.(ex:
> the timestamp for t0 should be mapping to offset 10, but will get offset 12..
> etc). It'll cause the time index is putting the wrong offset, so the result
> will be unexpected.
> ===
> 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)