KAFKA-16310 ListOffsets doesn't report the offset with maxTimestamp a…#15621
KAFKA-16310 ListOffsets doesn't report the offset with maxTimestamp a…#15621chia7712 merged 52 commits intoapache:trunkfrom
Conversation
junrao
left a comment
There was a problem hiding this comment.
@chia7712 : Thanks for the PR.
We also need to
(1) revert all offsetForMaxTimestamp to shallowOffsetMaxTimestamp
(2) change/revert the implementation to set shallowOffsetMaxTimestamp accordingly.
(3) add tests for follower appends
Do we need to revert all of them? the paths we had fixed works well now.
@junrao WDYT?
will complete it later |
| // case 0: test the offsets from leader's append path | ||
| check() | ||
|
|
||
| // case 1: test the offsets from follower's append path. |
There was a problem hiding this comment.
@junrao the extra tests are added. please take a look
@chia7712 : I think both are important. First, it's important to be able to derive the same thing consistently from the leader and the follower log. This affects things like the time indexing entries. It will be confusing if the leader adds an offset in the middle of a batch while the follower adds an offset at the end of the batch. Second, it's important to name things as accurately as possible. Otherwise, future developers could make inaccurate assumptions. |
you are right. I have reverted the impl and naming. Also, I add extra comments for the "spec" of offsetOfMaxTimestamp |
|
rebase code and apply Luke's patch from https://github.com/chia7712/kafka/pull/3/files |
|
@junrao thanks for all your reviews and patience. all comments are addressed |
|
@chia7712 : There are quite a few test failures on kafka.server.ListOffsetsRequestTest.testResponseIncludesLeaderEpoch(). |
yep, I have fixed it on my local. will update PR later. thanks for the reminder :) |
|
@chia7712 : Thanks for the updated PR. The code looks good to me. There were 50 failed tests. Is any of them related to the PR? If not, have they all been tracked? |
there are many timeout exception, and so I feel that could be caused by busy server. I will trigger QA again instead of creating a bunch of flaky issues |
https://issues.apache.org/jira/browse/KAFKA-16494 https://issues.apache.org/jira/browse/KAFKA-15898 https://issues.apache.org/jira/browse/KAFKA-15945 https://issues.apache.org/jira/browse/KAFKA-16383 https://issues.apache.org/jira/browse/KAFKA-16024 https://issues.apache.org/jira/browse/KAFKA-16495 https://issues.apache.org/jira/browse/KAFKA-15104 @junrao those failed tests pass on my local, and they have jira now. Please review this PR again. thanks! |
|
@chia7712 : Thanks for triaging the failed tests. In the last run, it seem that JDK 21 and Scala 2.13 didn't complete. Could you trigger another build? Typically, this could be done by closing the PR, waiting for 20 secs and opening it again. |
thanks for the tip. I rebase the code to trigger QA in order to make sure this PR works well with latest code :) |
https://issues.apache.org/jira/browse/KAFKA-16497 https://issues.apache.org/jira/browse/KAFKA-16383 https://issues.apache.org/jira/browse/KAFKA-15917 https://issues.apache.org/jira/browse/KAFKA-16498 https://issues.apache.org/jira/browse/KAFKA-15891 https://issues.apache.org/jira/browse/KAFKA-16499 https://issues.apache.org/jira/browse/KAFKA-16136 https://issues.apache.org/jira/browse/KAFKA-16500 https://issues.apache.org/jira/browse/KAFKA-15146 https://issues.apache.org/jira/browse/KAFKA-15927 https://issues.apache.org/jira/browse/KAFKA-14089 https://issues.apache.org/jira/browse/KAFKA-15146 https://issues.apache.org/jira/browse/KAFKA-16501 https://issues.apache.org/jira/browse/KAFKA-15963 https://issues.apache.org/jira/browse/KAFKA-16502 https://issues.apache.org/jira/browse/KAFKA-15104 ok, all pass on my local and they have jira. |
| val expectedOffsetOfMaxTimestamp = 1 | ||
| assertEquals(expectedOffsetOfMaxTimestamp, validatingResults.offsetOfMaxTimestampMs, | ||
| s"Offset of max timestamp should be 1") | ||
| assertEquals(2, validatingResults.shallowOffsetOfMaxTimestamp) |
There was a problem hiding this comment.
@chia7712 : There seems to be an existing bug. The method is checkNonCompressed(), but in line 370, we set the compression codec to GZIP.
There was a problem hiding this comment.
yep, that is a "TYPO" but it does not change the test. We do pass the "NONE" to create LogValidator so it will run the path assignOffsetsNonCompressed
However, I do observe a potential bug.
context
- Those batches can have different compression
- We take the compression from last batch
potential bug
topic-level compression = GZIP
batch_0 = NONE
batch_1 = GZIP
In this case, we don't rebuild records according to topic-level compression since the compression of "last batch" is equal to GZIP. Hence, it results in batch_0 having incorrect compression.
This bug does not produce corrupt records, so we can add comments/docs to describe that issue. Or we can fix it by changing the sourceCompression to be a "collection" of all batches' compression, and then do conversion if one of them is mismatched.
There was a problem hiding this comment.
for another, LogValidator is moved to storage module already but its unit test is still in core module. That is a bit weird. We can rewrite it by java with bug fix and then move it to storage module. I have filed https://issues.apache.org/jira/browse/KAFKA-16689
There was a problem hiding this comment.
yep, that is a "TYPO" but it does not change the test.
The issue is that the test is testing the wrong expected value. For magic of 1, the offset for max timestamp should be 1 instead of 2.
However, I do observe a potential bug.
Yes, this can lead to inaccurate LogAppendInfo.sourceCompression. But it doesn't seem to have real impact now. LogAppendInfo.sourceCompression is only used in LogValidator, which is only called by the leader. In the leader, currently, we expect only 1 batch per producer.
apache#15621) We do iterate the records to find the offsetOfMaxTimestamp instead of returning the cached one when handling ListOffsetsRequest.MAX_TIMESTAMP, since it is hard to align all paths to get correct offsetOfMaxTimestamp. The known paths are shown below. 1. convertAndAssignOffsetsNonCompressed -> we CAN get correct offsetOfMaxTimestamp when validating all records 2. assignOffsetsNonCompressed -> ditto 3. validateMessagesAndAssignOffsetsCompressed -> ditto 4. validateMessagesAndAssignOffsetsCompressed#buildRecordsAndAssignOffsets -> ditto 5. appendAsFollow#append#analyzeAndValidateRecords -> we CAN'T get correct offsetOfMaxTimestamp as iterating all records is expensive when fetching records from leader 6. LogSegment#recover -> ditto Reviewers: Jun Rao <junrao@gmail.com>
We do iterate the records to find the
offsetOfMaxTimestampinstead of returning the cached one when handlingListOffsetsRequest.MAX_TIMESTAMP, since it is hard to align all paths to get correctoffsetOfMaxTimestamp. The known paths are shown below.convertAndAssignOffsetsNonCompressed-> we CAN get correctoffsetOfMaxTimestampwhen validating all recordsassignOffsetsNonCompressed-> dittovalidateMessagesAndAssignOffsetsCompressed-> dittovalidateMessagesAndAssignOffsetsCompressed#buildRecordsAndAssignOffsets-> dittoappendAsFollow#append#analyzeAndValidateRecords-> we CAN'T get correctoffsetOfMaxTimestampas iterating all records is expensive when fetching records from leaderLogSegment#recover-> dittohttps://issues.apache.org/jira/browse/KAFKA-16310
Committer Checklist (excluded from commit message)