Skip to content

KAFKA-806: Index may not always observe log.index.interval.bytes#18012

Merged
junrao merged 9 commits intoapache:trunkfrom
FrankYang0529:KAFKA-806
Jan 17, 2025
Merged

KAFKA-806: Index may not always observe log.index.interval.bytes#18012
junrao merged 9 commits intoapache:trunkfrom
FrankYang0529:KAFKA-806

Conversation

@FrankYang0529
Copy link
Copy Markdown
Member

@FrankYang0529 FrankYang0529 commented Dec 3, 2024

Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Changes in this PR:

  • Iterate each batch in MemoryRecords. If bytes since last index entry is large than log.index.interval.bytes, appending a offset/timestamp index.
  • In before, if a LogSegment is new, we set rollingBasedTimestamp as largest timestamp in the first MemoryRecords. After this PR, rollingBasedTimestamp is largest timestamp in the first batch, so the behavior is consistent between leader and follower.
  • Remove LogAppendInfor#shallowOffsetOfMaxTimestamp. We iterate each batch in append function, so we can get the value by shallowOffsetOfMaxTimestampSoFar.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@github-actions github-actions Bot added storage Pull requests that target the storage module small Small PRs labels Dec 3, 2024
@FrankYang0529 FrankYang0529 force-pushed the KAFKA-806 branch 2 times, most recently from d21b5f6 to e8d2320 Compare December 3, 2024 11:50
@FrankYang0529 FrankYang0529 force-pushed the KAFKA-806 branch 2 times, most recently from a80cc4c to ebfb248 Compare December 7, 2024 05:41
@ijuma ijuma requested a review from junrao December 24, 2024 15:40
Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 : Thanks for the PR. Left a couple of comments.

for (RecordBatch batch : records.batches()) {
if (bytesSinceLastIndexEntry > indexIntervalBytes &&
batch.lastOffset() >= offsetIndex().lastOffset()) {
offsetIndex().append(batch.lastOffset(), physicalPosition);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, every time we append an entry to the offset index, we also append an entry to the timestamp index. It would be useful to keep it that way.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @junrao, thanks for review. I addressed both comments.

For timestamp, it's not always monotonic in records, so checking offset by timestamp index is not as much as offset index. Probably, we can consider whether it's worth to add timestamp for each batch, because this operation introduces more cost.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's true that the time index is used less frequently. But I am not sure it's worth optimizing since its size is small.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we can do a bit refactor for this hot method to reduce the access of buffer and check.

            for (RecordBatch batch : records.batches()) {
                var batchMaxTimestamp = batch.maxTimestamp();
                var batchLastOffset = batch.lastOffset();
                var updateTimeIndex = false;
                if (batchMaxTimestamp > maxTimestampSoFar()) {
                    maxTimestampAndOffsetSoFar = new TimestampOffset(batchMaxTimestamp, batchLastOffset);
                    updateTimeIndex = true;
                }

                if (bytesSinceLastIndexEntry > indexIntervalBytes) {
                    offsetIndex().append(batchLastOffset, physicalPosition);

                    // max timestamp may not be monotonic, so we need to check it to avoid the time index append error
                    if (updateTimeIndex) timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar());

                    bytesSinceLastIndexEntry = 0;
                }
                var sizeInBytes = batch.sizeInBytes();
                physicalPosition += sizeInBytes;
                bytesSinceLastIndexEntry += sizeInBytes;
            }

// append an entry to the offset index at batches level (if needed)
for (RecordBatch batch : records.batches()) {
if (bytesSinceLastIndexEntry > indexIntervalBytes &&
batch.lastOffset() >= offsetIndex().lastOffset()) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is unnecessary. On the leader side, we call LogValidator.validateBatch, which makes sure each batch has at least one record.

@github-actions github-actions Bot removed the small Small PRs label Jan 8, 2025
@FrankYang0529 FrankYang0529 force-pushed the KAFKA-806 branch 2 times, most recently from b9f1ae1 to e08001e Compare January 8, 2025 14:58
Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 : Thanks for the updated PR. A few more comments.

long batchShallowOffsetOfMaxTimestamp = -1L;
// append an entry to the index at batches level (if needed)
for (RecordBatch batch : records.batches()) {
if (batch.maxTimestamp() > batchMaxTimestamp) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we get rid of this condition and make batchMaxTimestamp and batchShallowOffsetOfMaxTimestamp a local val in the loop?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, updated it.

long appendedBytes = log.append(records);
LOGGER.trace("Appended {} to {} at end offset {}", appendedBytes, log.file(), largestOffset);
// Update the in memory max timestamp and corresponding offset.
if (largestTimestampMs > maxTimestampSoFar()) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With this change, we can remove both largestTimestampMs and shallowOffsetOfMaxTimestamp. We probably can also remove LogAppendInfo.shallowOffsetOfMaxTimestamp.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, removed largestTimestampMs and shallowOffsetOfMaxTimestamp in LogSegment#append. Also, removed LogAppendInfo.shallowOffsetOfMaxTimestamp because it's not referenced anymore.

offsetIndex().append(batch.lastOffset(), physicalPosition);

// max timestamp may not be monotonic, so we need to check it to avoid the time index append error
if (batchMaxTimestamp >= timeIndex().lastEntry().timestamp)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is unnecessary since TimeIndex has the same check.

            // We only append to the time index when the timestamp is greater than the last inserted timestamp.
            // If all the messages are in message format v0, the timestamp will always be NoTimestamp. In that case, the time
            // index will be empty.
            if (timestamp > lastEntry.timestamp) {

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think @FrankYang0529 tried to avoid the exception of adding a "smaller" timestamp. see

throw new IllegalStateException("Attempt to append a timestamp (" + timestamp + ") to slot " + entries()

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, like @chia7712 mentioned, if there is timestamp index in the file and new timestamp is less than old one, TimeIndex#maybeAppend will throw error.


// max timestamp may not be monotonic, so we need to check it to avoid the time index append error
if (batchMaxTimestamp >= timeIndex().lastEntry().timestamp)
timeIndex().maybeAppend(batchMaxTimestamp, shallowOffsetOfMaxTimestampSoFar());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, we should use maxTimestampSoFar() instead batchMaxTimestamp, right?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, updated it. Thanks.

for (RecordBatch batch : records.batches()) {
if (bytesSinceLastIndexEntry > indexIntervalBytes &&
batch.lastOffset() >= offsetIndex().lastOffset()) {
offsetIndex().append(batch.lastOffset(), physicalPosition);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's true that the time index is used less frequently. But I am not sure it's worth optimizing since its size is small.

@github-actions github-actions Bot added the core Kafka Broker label Jan 9, 2025
Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 : Thanks for the update PR. A few more comments.

}
bytesSinceLastIndexEntry += records.sizeInBytes();

if (updateRollingBasedTimestamp) rollingBasedTimestamp = OptionalLong.of(recordsLargestTimestampMs);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to assign the timestamp of the first batch in a segment to rollingBasedTimestamp. This way, it's consistent between the leader and the follower.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since we utilize the timestamp of the first batch in other parts of the code, it is beneficial to maintain consistency by aligning it within this PR. Alternatively, we can keep rollingBasedTimestamp empty and initialize it during the method call.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree, for consistency, we can leave LogSegment#loadFirstBatchTimestamp to handle rollingBasedTimestamp. Remove related code in append function.

new LogAppendInfo(firstOffset, lastOffset, lastLeaderEpochOpt, maxTimestamp, shallowOffsetOfMaxTimestamp,
RecordBatch.NO_TIMESTAMP, logStartOffset, RecordValidationStats.EMPTY, sourceCompression,
validBytesCount, lastOffsetOfFirstBatch, Collections.emptyList[RecordError], LeaderHwChange.NONE)
new LogAppendInfo(firstOffset, lastOffset, lastLeaderEpochOpt, maxTimestamp, RecordBatch.NO_TIMESTAMP, logStartOffset, RecordValidationStats.EMPTY, sourceCompression, validBytesCount, lastOffsetOfFirstBatch, Collections.emptyList[RecordError], LeaderHwChange.NONE)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's probably better to still split this long statement into multiple lines.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, updated it. Thanks.


seg.append(41, RecordBatch.NO_TIMESTAMP, -1L, v1Records(40, "hello", "there"));
seg.append(41,
MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 40, Compression.NONE, TimestampType.CREATE_TIME,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this change needed? Also, since we no longer support v0/v1 message format. We need to remove all v1Records usage in this test. Should we do it in this PR or a separate one?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In 4.0, we don't support v0/v1 message from producer. However, do we support a case that a 4.0 broker works with other versions and the leader node has old message format? In this case, we may still get v0/v1 message format.

Copy link
Copy Markdown
Member

@chia7712 chia7712 Jan 13, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think @FrankYang0529 is interested in testing a scenario where a 4.0 follower sends a FetchRequest to a leader that contains v0/v1 messages.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok. Why do we change from inserting 2 records to 1 record? Also, could we continue using v1Records?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we change from inserting 2 records to 1 record?

My mistake. Changed it back to 2 records.

could we continue using v1Records?

Probably no. The old test rely on largestTimestampMs to assign rollingBasedTimestamp as RecordBatch.NO_TIMESTAMP. In this PR, we assign first batch max timestamp for rollingBasedTimestamp. In v1Records, it sets record timestamp as offset * 10, so it's not RecordBatch.NO_TIMESTAMP.

seg.append(41, RecordBatch.NO_TIMESTAMP, -1L, v1Records(40, "hello", "there"));

private MemoryRecords v1Records(long offset, String... records) {
List<SimpleRecord> simpleRecords = new ArrayList<>();
for (String s : records) {
simpleRecords.add(new SimpleRecord(offset * 10, s.getBytes()));

boolean updateTimeIndex = false;
if (batchMaxTimestamp > maxTimestampSoFar()) {
maxTimestampAndOffsetSoFar = new TimestampOffset(batchMaxTimestamp, batchLastOffset);
updateTimeIndex = true;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This code seems still not quite right. We need to remember updateTimeIndex once we get a higher timestamp instead of resetting it in every batch.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, remove updateTimeIndex. Thanks.

offsetIndex().append(batchLastOffset, physicalPosition);

// max timestamp may not be monotonic, so we need to check it to avoid the time index append error
if (updateTimeIndex) timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that we don't need to maintain updateTimeIndex. maxTimestampSoFar() is always >= the last time index entry. If it's equal, it will be ignored in timeIndex().maybeAppend and won't trigger IllegalStateException.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The purpose of using updateTimeIndex is to minimize unnecessary calls to maybeAppend by ensuring that it's only invoked when a larger timestamp is encountered. That can avoid requiring unnecessary locking, but I think maybe that is unnecessary optimization. We can totally remove updateTimeIndex to simplify code

expectedMaxTimestampOffset = 2;
break;
}
assertEquals(expectedMaxTimestampOffset, validatedResults.shallowOffsetOfMaxTimestamp);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

expectedMaxTimestampOffset is no longer used and should be removed.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed it. Thanks.

assertEquals(1, segment.offsetIndex().lookup(1L).offset);
assertEquals(2, segment.offsetIndex().lookup(2L).offset);

assertEquals(1, segment.timeIndex().entries());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that we should have two index entries in this test, the first for timestamp 1 and the second of timestamp 2.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, updated it.


assertEquals(1, segment.timeIndex().entries());
assertEquals(0L, segment.timeIndex().lookup(1L).offset);
assertEquals(2L, segment.timeIndex().lookup(2L).offset);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we use timeIndex().entry to explicitly fetch each index entry instead of using timeIndex().lookup?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated assertion. Thanks.

Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 : Thanks for the updated PR. A few more comments.

if (bytesSinceLastIndexEntry > indexIntervalBytes) {
offsetIndex().append(batchLastOffset, physicalPosition);
timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar());

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

extra newline

assertEquals(2, segment.offsetIndex().lookup(2L).offset);

assertEquals(2, segment.timeIndex().entries());
assertEquals(new TimestampOffset(1, 0), segment.timeIndex().entry(0));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we use entry for the offset index and the other test too?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the suggestion. I only check offset field for offset index, because the position field may looks like some kind of magic value. WDYT?


seg.append(41, RecordBatch.NO_TIMESTAMP, -1L, v1Records(40, "hello", "there"));
seg.append(41,
MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 40, Compression.NONE, TimestampType.CREATE_TIME,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok. Why do we change from inserting 2 records to 1 record? Also, could we continue using v1Records?

@FrankYang0529 FrankYang0529 force-pushed the KAFKA-806 branch 3 times, most recently from bed1b2b to 717f252 Compare January 14, 2025 05:01
Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 : Thanks for the updated PR. LGTM. Are the flaky tests related to this PR?

@FrankYang0529
Copy link
Copy Markdown
Member Author

Hi @junrao, thanks for the review. I check failed case and all are not related to this PR. I can't reproduce them on my laptop.

@FrankYang0529
Copy link
Copy Markdown
Member Author

Last build: https://github.com/apache/kafka/actions/runs/12784403901/job/35637865171?pr=18012

Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 thanks for this patch. please take a look at following two minor questions

now,
records,
maxTimestamp,
shallowOffsetOfMaxTimestamp,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shallowOffsetOfMaxTimestamp is useless, so could you please remove it also?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed it. Thanks for the reminder.


if (bytesSinceLastIndexEntry > indexIntervalBytes) {
offsetIndex().append(batchLastOffset, physicalPosition);
timeIndex().maybeAppend(maxTimestampSoFar(), shallowOffsetOfMaxTimestampSoFar());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If two batches have the same timestamp, and both timestamps are greater than maxTimestampSoFar, and the first batch is skipped due to indexIntervalBytes, the resulting index for searching the 'earliest offset' of the maximum timestamp using TimestampSpec will be incorrect.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, in that case, maxTimestampAndOffsetSoFar will point to the offset of the first batch, which will be appended to the time index. This seem correct, right?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you are right, we update the maxTimestampAndOffsetSoFar only if batchMaxTimestamp > maxTimestampSoFar() is true. Hence, the offset point to the first batch.

Signed-off-by: PoAn Yang <payang@apache.org>
Signed-off-by: PoAn Yang <payang@apache.org>
Signed-off-by: PoAn Yang <payang@apache.org>
Signed-off-by: PoAn Yang <payang@apache.org>
Signed-off-by: PoAn Yang <payang@apache.org>
Signed-off-by: PoAn Yang <payang@apache.org>
Signed-off-by: PoAn Yang <payang@apache.org>
Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 : Thanks for the updated PR. One more minor comment.

MetricsRecorder metricsRecorder) {
long now = time.milliseconds();
long maxTimestamp = RecordBatch.NO_TIMESTAMP;
long shallowOffsetOfMaxTimestamp = -1L;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we also remove initialOffset?

Signed-off-by: PoAn Yang <payang@apache.org>
Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 : Thanks for the updated PR. LGTM

@junrao junrao merged commit e124d39 into apache:trunk Jan 17, 2025
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 17, 2025

Can we please update the PR description with what we decided to do here? It's a bit sparse.

@FrankYang0529 FrankYang0529 deleted the KAFKA-806 branch January 18, 2025 01:53
@FrankYang0529
Copy link
Copy Markdown
Member Author

@junrao @chia7712, thanks for the review.

@ijuma, updated the PR description. Thanks.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jan 18, 2025

Thanks! The last bullet point seems to be missing a word or something.

pranavt84 pushed a commit to pranavt84/kafka that referenced this pull request Jan 27, 2025
…che#18012)

Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
airlock-confluentinc Bot pushed a commit to confluentinc/kafka that referenced this pull request Jan 27, 2025
…che#18012)

Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
}
// append an entry to the index (if needed)
if (bytesSinceLastIndexEntry > indexIntervalBytes) {
offsetIndex().append(largestOffset, physicalPosition);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In scenarios where a new follower is synchronizing from another replica, another issue arises. If the records consists of multiple batches, the physicalPosition will reflect the position of the first batch. This leads to an inaccurate index, represented as (offset_of_last_batch, position_of_first_batch). This issue can be easily reproduced, and our dump log tool currently displays warnings to indicate this inconsistency.

Mismatches in :/home/chia7712/ikea-0-follower/00000000000000000000.index
  Index offset: 2062680, log offset: 2061012
  Index offset: 2060984, log offset: 2058974
  Index offset: 2058940, log offset: 2056966
  Index offset: 2056936, log offset: 2054979
  Index offset: 2054950, log offset: 2052962
  Index offset: 2052931, log offset: 2050959
  Index offset: 2050923, log offset: 2048924
  Index offset: 2048895, log offset: 2046967

This drawback is analogous to having fewer index entries, which can lead to increased fetch times. I'm happy that @FrankYang0529 fix "two" issues by this PR.

However, I'm wondering if we should address this issue in the active 3.x version. We could calculate the accurate position by using the size of the last batch. For example:

                int position = records.lastBatch().map(b -> log.sizeInBytes() - b.sizeInBytes()).orElse(physicalPosition);
                offsetIndex().append(largestOffset, position);

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chia7712 Thanks for the information. I will create a PR to 3.9 tomorrow.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 I open https://issues.apache.org/jira/browse/KAFKA-18759 to trace it for 3.9 and 3.8 - please feel free to take over this jira

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chia7712 : If we want to address the indexing issue in 3.9, it's probably easier to just cherry-pick this PR to 3.9 instead of fixing a subset of the problem separately, right?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@junrao I tried to minimize the changes to the 3.9, but you are right that we can backport this PR instead.

@FrankYang0529 WDYT? If you agree to backport this PR, could you please file a PR for 3.9? Thanks.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

4.0: #18842
3.9: #18843

FrankYang0529 added a commit to FrankYang0529/kafka that referenced this pull request Feb 8, 2025
…che#18012)

Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
(cherry picked from commit e124d39)
FrankYang0529 added a commit to FrankYang0529/kafka that referenced this pull request Feb 8, 2025
…che#18012)

Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
(cherry picked from commit e124d39)
manoj-mathivanan pushed a commit to manoj-mathivanan/kafka that referenced this pull request Feb 19, 2025
…che#18012)

Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
FrankYang0529 added a commit to FrankYang0529/kafka that referenced this pull request Mar 20, 2025
…che#18012)

Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
(cherry picked from commit e124d39)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Kafka Broker storage Pull requests that target the storage module

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants