KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP#15213
KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP#15213showuon merged 3 commits intoapache:trunkfrom
Conversation
| val earliestLocalLogEpochEntry = leaderEpochCache.asJava.flatMap(cache => { | ||
| val epochOpt: Optional[Integer] = leaderEpochCache.asJava.flatMap(cache => { | ||
| val epoch = cache.epochForOffset(curLocalLogStartOffset) | ||
| if (epoch.isPresent) cache.epochEntry(epoch.getAsInt) else Optional.empty[EpochEntry]() | ||
| if (epoch.isPresent) Optional.of[Integer](epoch.getAsInt) else Optional.empty[Integer]() | ||
| }) | ||
|
|
||
| val epochOpt = if (earliestLocalLogEpochEntry.isPresent && earliestLocalLogEpochEntry.get().startOffset <= curLocalLogStartOffset) | ||
| Optional.of[Integer](earliestLocalLogEpochEntry.get().epoch) | ||
| else Optional.empty[Integer]() | ||
|
|
There was a problem hiding this comment.
I didn't really see a point in this check earliestLocalLogEpochEntry.get().startOffset <= curLocalLogStartOffset. As far as I can tell the cache.epochForOffset already carries it out. Let me know in case I have misunderstood something.
There was a problem hiding this comment.
This is done for backward compatibility with older message format. You can go through the comments in the previous block:
The first cached epoch usually corresponds to the log start offset, but we have to verify this since it may not be true following a message format version bump as the epoch will not be available for log entries written in the older format.
There was a problem hiding this comment.
subjective:
Can we update the method similar to below for readability/clarity:
if (remoteLogEnabled()) {
val curHighestRemoteOffset = highestOffsetInRemoteStorage()
var result: Optional[Integer] = Optional.of(RecordBatch.NO_PARTITION_LEADER_EPOCH)
if (leaderEpochCache.isDefined) {
val epochOpt = leaderEpochCache.get.epochForOffset(curHighestRemoteOffset)
if (epochOpt.isPresent) {
result = Optional.of(epochOpt.getAsInt)
}
}
Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, highestOffsetInRemoteStorage(), result))
}Note when the highestOffsetInRemoteStorage is -1, then there won't be corresponding leaderEpoch, we have to return the NO_PARTITION_LEADER_EPOCH constant which is -1.
There was a problem hiding this comment.
Hopefully I have achieved both in the subsequent commit (reverted one of the changes and made both easier to read). Let me know if this isn't the case!
| } | ||
|
|
||
| @Test | ||
| def testFetchLatestTieredTimestampWithRemoteStorage(): Unit = { |
There was a problem hiding this comment.
This test could be combined with testFetchOffsetByTimestampFromRemoteStorage as the only difference it has are lines 2167, 2193, 2203 and 2204. Let me know your thoughts!
There was a problem hiding this comment.
Let's maintain two tests for clarity. Can you add comments for L2197 log.fetchOffsetByTimestamp(secondTimestamp, Some(remoteLogManager)), that it search for offset from both remote and local storage?
| targetTimestamp != ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP && | ||
| targetTimestamp != ListOffsetsRequest.LATEST_TIMESTAMP) | ||
| targetTimestamp != ListOffsetsRequest.LATEST_TIMESTAMP && | ||
| targetTimestamp != ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) |
There was a problem hiding this comment.
Should we allow the timestamp -4 and -5 when IBP is less than 0.10? Tiered Storage is supported from IBP 2.8-IV1
There was a problem hiding this comment.
I have removed them both, but I don't think it would have caused problems either way.
| val earliestLocalLogEpochEntry = leaderEpochCache.asJava.flatMap(cache => { | ||
| val epochOpt: Optional[Integer] = leaderEpochCache.asJava.flatMap(cache => { | ||
| val epoch = cache.epochForOffset(curLocalLogStartOffset) | ||
| if (epoch.isPresent) cache.epochEntry(epoch.getAsInt) else Optional.empty[EpochEntry]() | ||
| if (epoch.isPresent) Optional.of[Integer](epoch.getAsInt) else Optional.empty[Integer]() | ||
| }) | ||
|
|
||
| val epochOpt = if (earliestLocalLogEpochEntry.isPresent && earliestLocalLogEpochEntry.get().startOffset <= curLocalLogStartOffset) | ||
| Optional.of[Integer](earliestLocalLogEpochEntry.get().epoch) | ||
| else Optional.empty[Integer]() | ||
|
|
There was a problem hiding this comment.
This is done for backward compatibility with older message format. You can go through the comments in the previous block:
The first cached epoch usually corresponds to the log start offset, but we have to verify this since it may not be true following a message format version bump as the epoch will not be available for log entries written in the older format.
|
|
||
| Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, highestOffsetInRemoteStorage(), optEpoch)) | ||
| } else { | ||
| Option.empty |
There was a problem hiding this comment.
If we return empty, then exception here will be thrown back to the caller. Do we want to throw an error back to the caller (or) empty response?
Some(new TimestampAndOffset(RecordBatch.NO_TIMESTAMP, -1L, Optional.of(-1))
There was a problem hiding this comment.
Good catch! In the KIP I have specified that Kafka should return no offset in such situations. I shall aim to add an integration test from the point of view of the client in an upcoming pull request
|
|
||
| log.appendAsLeader(TestUtils.singletonRecords( | ||
| value = TestUtils.randomBytes(10), | ||
| timestamp = firstTimestamp), |
There was a problem hiding this comment.
Why are we using the first timestamp again?
There was a problem hiding this comment.
There is nothing preventing people to send batches of records with out-of-order timestamps (in fact, entries in the timeindex take this into account). I also think that for this particular case it doesn't matter as what I am trying to test here is that we don't get a response back. For the sake of focusing on what the test is supposed to test, however, I have removed this 😊
| } | ||
|
|
||
| @Test | ||
| def testFetchLatestTieredTimestampWithRemoteStorage(): Unit = { |
There was a problem hiding this comment.
Let's maintain two tests for clarity. Can you add comments for L2197 log.fetchOffsetByTimestamp(secondTimestamp, Some(remoteLogManager)), that it search for offset from both remote and local storage?
5d62284 to
791f786
Compare
|
Thanks a lot for the review @kamalcph! I have hopefully addressed everything 😊 |
kamalcph
left a comment
There was a problem hiding this comment.
LGTM, thanks for the patch!
We can add integration tests once we add support to query the latest-tiered and earliest-local offsets from the admin-client.
|
Will check this this or next week. Thanks. |
showuon
left a comment
There was a problem hiding this comment.
Thanks for the PR. Left some comments.
|
|
||
| if (config.messageFormatVersion.isLessThan(IBP_0_10_0_IV0) && | ||
| targetTimestamp != ListOffsetsRequest.EARLIEST_TIMESTAMP && | ||
| targetTimestamp != ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP && |
There was a problem hiding this comment.
Correct, Kamal called this out
| else Optional.empty[Integer]() | ||
| var epochResult: Optional[Integer] = Optional.of(RecordBatch.NO_PARTITION_LEADER_EPOCH) |
There was a problem hiding this comment.
Why do we change the return for "no leaderEpoch" case from empty to -1? I had a check, it seems won't change anything because the default value of leaderEpoch in ListOffsetsPartitionResponse is -1. Any thought on this change?
There was a problem hiding this comment.
You are correct, this is a miss on my side as part of making this piece of code more readable - fixing it in the subsequent commit.
The logic should be
- For EARLIEST_LOCAL_TIMESTAMP - return empty unless there is a leader epoch
- For LATEST_TIERED_TIMESTAMP - if highest offset is -1 then return -1, if there is a highest offset then return empty unless there is a leader epoch
|
I will take a look at this PR by |
791f786 to
4fa7671
Compare
4fa7671 to
af4eb14
Compare
|
@clolov , there's a compilation error in jdk 8/scala 2.12. Could you help fix it? Maybe rebasing to the latest trunk could solve it? |
|
@clolov , do we have any update about the compilation error fix? |
|
Apologies, I have had a lot of work recently, I will aim to provide an update by the end of the day today |
af4eb14 to
ac20141
Compare
|
Heya @showuon, I tried to reproduce the compilation failure of the unrelated test class locally, but have been unable to. I have rebased and kicked off a new build in case this was a temporary failure. If it occurs again I will continue looking into it. |
|
Failed tests are unrelated. |
| */ | ||
| public static final long EARLIEST_LOCAL_TIMESTAMP = -4L; | ||
|
|
||
| public static final long LATEST_TIERED_TIMESTAMP = -5L; |
There was a problem hiding this comment.
Typically, if we add a new targetTimestamp value, we will need to bump up the version of the ListOffsetsRequest. See https://github.com/apache/kafka/pull/10760/files. Otherwise, a client could be setting LATEST_TIERED_TIMESTAMP and assuming that the server supports it, but the server actually does not.
There was a problem hiding this comment.
Yup, thanks a lot for bringing this up in the mailing list and here, I will open a pull request to amend this miss!
…e#15213) This is the first part of the implementation of KIP-1005 The purpose of this pull request is for the broker to start returning the correct offset when it receives a -5 as a timestamp in a ListOffsets API request Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>
…e#15213) This is the first part of the implementation of KIP-1005 The purpose of this pull request is for the broker to start returning the correct offset when it receives a -5 as a timestamp in a ListOffsets API request Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>
apache#15213)" This reverts commit 55a6d30
apache#15213)" This reverts commit 55a6d30
…6400) * Revert "KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP (#15213)" * Set 3.8_IV0 as latest production version in 3.8 * Bring changes committed in KAFKA-16968 * Make ClusterTest annotation metadata default to 3.9 --------- Signed-off-by: Josep Prat <josep.prat@aiven.io> Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
Summary
This is the first part of the implementation of KIP-1005
The purpose of this pull request is for the broker to start returning the correct offset when it receives a -5 as a timestamp in a ListOffsets API request