junrao commented on code in PR #18012:
URL: https://github.com/apache/kafka/pull/18012#discussion_r1909240447
##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1180,9 +1179,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
else
OptionalInt.empty()
- 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)
Review Comment:
It's probably better to still split this long statement into multiple lines.
##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java:
##########
@@ -232,38 +232,48 @@ private boolean canConvertToRelativeOffset(long offset)
throws IOException {
* It is assumed this method is being called from within a lock, it is not
thread-safe otherwise.
*
* @param largestOffset The last offset in the message set
- * @param largestTimestampMs The largest timestamp in the message set.
- * @param shallowOffsetOfMaxTimestamp The last offset of earliest batch
with max timestamp in the messages to append.
- * @param records The log entries to append.
+ * @param records The log entries to append.
* @throws LogSegmentOffsetOverflowException if the largest offset causes
index offset overflow
*/
public void append(long largestOffset,
- long largestTimestampMs,
- long shallowOffsetOfMaxTimestamp,
MemoryRecords records) throws IOException {
if (records.sizeInBytes() > 0) {
- LOGGER.trace("Inserting {} bytes at end offset {} at position {}
with largest timestamp {} at offset {}",
- records.sizeInBytes(), largestOffset, log.sizeInBytes(),
largestTimestampMs, shallowOffsetOfMaxTimestamp);
+ LOGGER.trace("Inserting {} bytes at end offset {} at position {}",
+ records.sizeInBytes(), largestOffset, log.sizeInBytes());
int physicalPosition = log.sizeInBytes();
- if (physicalPosition == 0)
- rollingBasedTimestamp = OptionalLong.of(largestTimestampMs);
+ boolean updateRollingBasedTimestamp = physicalPosition == 0;
ensureOffsetInRange(largestOffset);
// append the messages
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()) {
- maxTimestampAndOffsetSoFar = new
TimestampOffset(largestTimestampMs, shallowOffsetOfMaxTimestamp);
- }
- // append an entry to the index (if needed)
- if (bytesSinceLastIndexEntry > indexIntervalBytes) {
- offsetIndex().append(largestOffset, physicalPosition);
- timeIndex().maybeAppend(maxTimestampSoFar(),
shallowOffsetOfMaxTimestampSoFar());
- bytesSinceLastIndexEntry = 0;
+
+ long recordsLargestTimestampMs = RecordBatch.NO_TIMESTAMP;
+ for (RecordBatch batch : records.batches()) {
+ long batchMaxTimestamp = batch.maxTimestamp();
+ long batchLastOffset = batch.lastOffset();
+ recordsLargestTimestampMs =
Math.max(recordsLargestTimestampMs, batchMaxTimestamp);
+ boolean 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;
}
- bytesSinceLastIndexEntry += records.sizeInBytes();
+
+ if (updateRollingBasedTimestamp) rollingBasedTimestamp =
OptionalLong.of(recordsLargestTimestampMs);
Review Comment:
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.
##########
storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java:
##########
@@ -342,7 +344,9 @@ public void testTruncateFull() throws IOException {
MockTime time = new MockTime();
try (LogSegment seg = createSegment(40, time)) {
- 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,
Review Comment:
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?
##########
storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java:
##########
@@ -2087,7 +2076,6 @@ public void checkLogAppendTimeNonCompressed(byte magic) {
expectedMaxTimestampOffset = 2;
break;
}
- assertEquals(expectedMaxTimestampOffset,
validatedResults.shallowOffsetOfMaxTimestamp);
Review Comment:
`expectedMaxTimestampOffset` is no longer used and should be removed.
##########
storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java:
##########
@@ -779,6 +784,75 @@ public void testDeleteIfExistsWithGetParentIsNull() throws
IOException {
}
}
+ @Test
+ public void testIndexForMultipleBatchesInMemoryRecords() throws
IOException {
+ LogSegment segment = createSegment(0, 1, Time.SYSTEM);
+
+ ByteBuffer buffer1 = ByteBuffer.allocate(1024);
+ // append first batch to buffer1
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer1,
Compression.NONE, TimestampType.CREATE_TIME, 0);
+ builder.append(0L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ // append second batch to buffer1
+ builder = MemoryRecords.builder(buffer1, Compression.NONE,
TimestampType.CREATE_TIME, 1);
+ builder.append(1L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ buffer1.flip();
+ MemoryRecords record = MemoryRecords.readableRecords(buffer1);
+ segment.append(1L, record);
+
+ ByteBuffer buffer2 = ByteBuffer.allocate(1024);
+ // append first batch to buffer2
+ builder = MemoryRecords.builder(buffer2, Compression.NONE,
TimestampType.CREATE_TIME, 2);
+ builder.append(2L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ buffer2.flip();
+ record = MemoryRecords.readableRecords(buffer2);
+ segment.append(2L, record);
+
+ assertEquals(2, segment.offsetIndex().entries());
+ assertTrue(segment.offsetIndex().lookup(2L).position >
segment.offsetIndex().lookup(1L).position);
+
+ assertEquals(2, segment.timeIndex().entries());
+ assertTrue(segment.timeIndex().lookup(2L).offset >
segment.timeIndex().lookup(1L).offset);
+ }
+
+ @Test
+ public void testNonMonotonicTimestampForMultipleBatchesInMemoryRecords()
throws IOException {
+ LogSegment segment = createSegment(0, 1, Time.SYSTEM);
+
+ ByteBuffer buffer1 = ByteBuffer.allocate(1024);
+ // append first batch to buffer1
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer1,
Compression.NONE, TimestampType.CREATE_TIME, 0);
+ builder.append(1L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ // append second batch to buffer1
+ builder = MemoryRecords.builder(buffer1, Compression.NONE,
TimestampType.CREATE_TIME, 1);
+ builder.append(0L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ // append third batch to buffer1
+ builder = MemoryRecords.builder(buffer1, Compression.NONE,
TimestampType.CREATE_TIME, 2);
+ builder.append(2L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ buffer1.flip();
+ MemoryRecords record = MemoryRecords.readableRecords(buffer1);
+ segment.append(2L, record);
+
+ assertEquals(2, segment.offsetIndex().entries());
+ assertEquals(1, segment.offsetIndex().lookup(1L).offset);
+ assertEquals(2, segment.offsetIndex().lookup(2L).offset);
+
+ assertEquals(1, segment.timeIndex().entries());
+ assertEquals(0L, segment.timeIndex().lookup(1L).offset);
+ assertEquals(2L, segment.timeIndex().lookup(2L).offset);
Review Comment:
Could we use `timeIndex().entry` to explicitly fetch each index entry
instead of using `timeIndex().lookup`?
##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java:
##########
@@ -232,38 +232,48 @@ private boolean canConvertToRelativeOffset(long offset)
throws IOException {
* It is assumed this method is being called from within a lock, it is not
thread-safe otherwise.
*
* @param largestOffset The last offset in the message set
- * @param largestTimestampMs The largest timestamp in the message set.
- * @param shallowOffsetOfMaxTimestamp The last offset of earliest batch
with max timestamp in the messages to append.
- * @param records The log entries to append.
+ * @param records The log entries to append.
* @throws LogSegmentOffsetOverflowException if the largest offset causes
index offset overflow
*/
public void append(long largestOffset,
- long largestTimestampMs,
- long shallowOffsetOfMaxTimestamp,
MemoryRecords records) throws IOException {
if (records.sizeInBytes() > 0) {
- LOGGER.trace("Inserting {} bytes at end offset {} at position {}
with largest timestamp {} at offset {}",
- records.sizeInBytes(), largestOffset, log.sizeInBytes(),
largestTimestampMs, shallowOffsetOfMaxTimestamp);
+ LOGGER.trace("Inserting {} bytes at end offset {} at position {}",
+ records.sizeInBytes(), largestOffset, log.sizeInBytes());
int physicalPosition = log.sizeInBytes();
- if (physicalPosition == 0)
- rollingBasedTimestamp = OptionalLong.of(largestTimestampMs);
+ boolean updateRollingBasedTimestamp = physicalPosition == 0;
ensureOffsetInRange(largestOffset);
// append the messages
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()) {
- maxTimestampAndOffsetSoFar = new
TimestampOffset(largestTimestampMs, shallowOffsetOfMaxTimestamp);
- }
- // append an entry to the index (if needed)
- if (bytesSinceLastIndexEntry > indexIntervalBytes) {
- offsetIndex().append(largestOffset, physicalPosition);
- timeIndex().maybeAppend(maxTimestampSoFar(),
shallowOffsetOfMaxTimestampSoFar());
- bytesSinceLastIndexEntry = 0;
+
+ long recordsLargestTimestampMs = RecordBatch.NO_TIMESTAMP;
+ for (RecordBatch batch : records.batches()) {
+ long batchMaxTimestamp = batch.maxTimestamp();
+ long batchLastOffset = batch.lastOffset();
+ recordsLargestTimestampMs =
Math.max(recordsLargestTimestampMs, batchMaxTimestamp);
+ boolean updateTimeIndex = false;
+ if (batchMaxTimestamp > maxTimestampSoFar()) {
+ maxTimestampAndOffsetSoFar = new
TimestampOffset(batchMaxTimestamp, batchLastOffset);
+ updateTimeIndex = true;
Review Comment:
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.
##########
storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java:
##########
@@ -779,6 +784,75 @@ public void testDeleteIfExistsWithGetParentIsNull() throws
IOException {
}
}
+ @Test
+ public void testIndexForMultipleBatchesInMemoryRecords() throws
IOException {
+ LogSegment segment = createSegment(0, 1, Time.SYSTEM);
+
+ ByteBuffer buffer1 = ByteBuffer.allocate(1024);
+ // append first batch to buffer1
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer1,
Compression.NONE, TimestampType.CREATE_TIME, 0);
+ builder.append(0L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ // append second batch to buffer1
+ builder = MemoryRecords.builder(buffer1, Compression.NONE,
TimestampType.CREATE_TIME, 1);
+ builder.append(1L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ buffer1.flip();
+ MemoryRecords record = MemoryRecords.readableRecords(buffer1);
+ segment.append(1L, record);
+
+ ByteBuffer buffer2 = ByteBuffer.allocate(1024);
+ // append first batch to buffer2
+ builder = MemoryRecords.builder(buffer2, Compression.NONE,
TimestampType.CREATE_TIME, 2);
+ builder.append(2L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ buffer2.flip();
+ record = MemoryRecords.readableRecords(buffer2);
+ segment.append(2L, record);
+
+ assertEquals(2, segment.offsetIndex().entries());
+ assertTrue(segment.offsetIndex().lookup(2L).position >
segment.offsetIndex().lookup(1L).position);
+
+ assertEquals(2, segment.timeIndex().entries());
+ assertTrue(segment.timeIndex().lookup(2L).offset >
segment.timeIndex().lookup(1L).offset);
+ }
+
+ @Test
+ public void testNonMonotonicTimestampForMultipleBatchesInMemoryRecords()
throws IOException {
+ LogSegment segment = createSegment(0, 1, Time.SYSTEM);
+
+ ByteBuffer buffer1 = ByteBuffer.allocate(1024);
+ // append first batch to buffer1
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer1,
Compression.NONE, TimestampType.CREATE_TIME, 0);
+ builder.append(1L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ // append second batch to buffer1
+ builder = MemoryRecords.builder(buffer1, Compression.NONE,
TimestampType.CREATE_TIME, 1);
+ builder.append(0L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ // append third batch to buffer1
+ builder = MemoryRecords.builder(buffer1, Compression.NONE,
TimestampType.CREATE_TIME, 2);
+ builder.append(2L, "key1".getBytes(), "value1".getBytes());
+ builder.close();
+
+ buffer1.flip();
+ MemoryRecords record = MemoryRecords.readableRecords(buffer1);
+ segment.append(2L, record);
+
+ assertEquals(2, segment.offsetIndex().entries());
+ assertEquals(1, segment.offsetIndex().lookup(1L).offset);
+ assertEquals(2, segment.offsetIndex().lookup(2L).offset);
+
+ assertEquals(1, segment.timeIndex().entries());
Review Comment:
It seems that we should have two index entries in this test, the first for
timestamp 1 and the second of timestamp 2.
##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java:
##########
@@ -232,38 +232,48 @@ private boolean canConvertToRelativeOffset(long offset)
throws IOException {
* It is assumed this method is being called from within a lock, it is not
thread-safe otherwise.
*
* @param largestOffset The last offset in the message set
- * @param largestTimestampMs The largest timestamp in the message set.
- * @param shallowOffsetOfMaxTimestamp The last offset of earliest batch
with max timestamp in the messages to append.
- * @param records The log entries to append.
+ * @param records The log entries to append.
* @throws LogSegmentOffsetOverflowException if the largest offset causes
index offset overflow
*/
public void append(long largestOffset,
- long largestTimestampMs,
- long shallowOffsetOfMaxTimestamp,
MemoryRecords records) throws IOException {
if (records.sizeInBytes() > 0) {
- LOGGER.trace("Inserting {} bytes at end offset {} at position {}
with largest timestamp {} at offset {}",
- records.sizeInBytes(), largestOffset, log.sizeInBytes(),
largestTimestampMs, shallowOffsetOfMaxTimestamp);
+ LOGGER.trace("Inserting {} bytes at end offset {} at position {}",
+ records.sizeInBytes(), largestOffset, log.sizeInBytes());
int physicalPosition = log.sizeInBytes();
- if (physicalPosition == 0)
- rollingBasedTimestamp = OptionalLong.of(largestTimestampMs);
+ boolean updateRollingBasedTimestamp = physicalPosition == 0;
ensureOffsetInRange(largestOffset);
// append the messages
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()) {
- maxTimestampAndOffsetSoFar = new
TimestampOffset(largestTimestampMs, shallowOffsetOfMaxTimestamp);
- }
- // append an entry to the index (if needed)
- if (bytesSinceLastIndexEntry > indexIntervalBytes) {
- offsetIndex().append(largestOffset, physicalPosition);
- timeIndex().maybeAppend(maxTimestampSoFar(),
shallowOffsetOfMaxTimestampSoFar());
- bytesSinceLastIndexEntry = 0;
+
+ long recordsLargestTimestampMs = RecordBatch.NO_TIMESTAMP;
+ for (RecordBatch batch : records.batches()) {
+ long batchMaxTimestamp = batch.maxTimestamp();
+ long batchLastOffset = batch.lastOffset();
+ recordsLargestTimestampMs =
Math.max(recordsLargestTimestampMs, batchMaxTimestamp);
+ boolean 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());
Review Comment:
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.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]