You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ch...@apache.org on 2024/03/28 20:40:47 UTC
(kafka) branch 3.7 updated: Revert "KAFKA-16341 fix the LogValidator for non-compressed type (#15476)"
This is an automated email from the ASF dual-hosted git repository.
chia7712 pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/3.7 by this push:
new bd5989dd195 Revert "KAFKA-16341 fix the LogValidator for non-compressed type (#15476)"
bd5989dd195 is described below
commit bd5989dd195d42c1608582316367a03b2c78cb11
Author: Chia-Ping Tsai <ch...@gmail.com>
AuthorDate: Fri Mar 29 04:37:53 2024 +0800
Revert "KAFKA-16341 fix the LogValidator for non-compressed type (#15476)"
This reverts commit f24228c4ffb7e61bc56b247873ef02da7edaba2d.
---
.../apache/kafka/common/record/MemoryRecords.java | 22 +++++------
.../kafka/common/record/MemoryRecordsBuilder.java | 6 +--
.../common/record/MemoryRecordsBuilderTest.java | 10 ++---
.../kafka/common/record/MemoryRecordsTest.java | 6 +--
core/src/main/scala/kafka/log/LocalLog.scala | 4 +-
core/src/main/scala/kafka/log/LogCleaner.scala | 2 +-
core/src/main/scala/kafka/log/UnifiedLog.scala | 2 +-
.../kafka/admin/ListOffsetsIntegrationTest.scala | 45 +++++++---------------
.../test/scala/unit/kafka/log/LocalLogTest.scala | 2 +-
.../test/scala/unit/kafka/log/LogSegmentTest.scala | 4 +-
.../scala/unit/kafka/log/LogValidatorTest.scala | 30 +++++++++------
.../kafka/storage/internals/log/LogSegment.java | 10 ++---
.../kafka/storage/internals/log/LogValidator.java | 16 +++++---
13 files changed, 75 insertions(+), 84 deletions(-)
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index c990a0a8043..888bcdf2cb1 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -211,7 +211,7 @@ public class MemoryRecords extends AbstractRecords {
partition, batch.lastOffset(), maxRecordBatchSize, filteredBatchSize);
MemoryRecordsBuilder.RecordsInfo info = builder.info();
- filterResult.updateRetainedBatchMetadata(info.maxTimestamp, info.offsetOfMaxTimestamp,
+ filterResult.updateRetainedBatchMetadata(info.maxTimestamp, info.shallowOffsetOfMaxTimestamp,
maxOffset, retainedRecords.size(), filteredBatchSize);
}
} else if (batchRetention == BatchRetention.RETAIN_EMPTY) {
@@ -400,7 +400,7 @@ public class MemoryRecords extends AbstractRecords {
private int bytesRetained = 0;
private long maxOffset = -1L;
private long maxTimestamp = RecordBatch.NO_TIMESTAMP;
- private long offsetOfMaxTimestamp = -1L;
+ private long shallowOffsetOfMaxTimestamp = -1L;
private FilterResult(ByteBuffer outputBuffer) {
this.outputBuffer = outputBuffer;
@@ -412,21 +412,21 @@ public class MemoryRecords extends AbstractRecords {
retainedBatch.lastOffset(), numMessagesInBatch, bytesRetained);
}
- private void updateRetainedBatchMetadata(long maxTimestamp, long offsetOfMaxTimestamp, long maxOffset,
- int messagesRetained, int bytesRetained) {
- validateBatchMetadata(maxTimestamp, offsetOfMaxTimestamp, maxOffset);
+ private void updateRetainedBatchMetadata(long maxTimestamp, long shallowOffsetOfMaxTimestamp, long maxOffset,
+ int messagesRetained, int bytesRetained) {
+ validateBatchMetadata(maxTimestamp, shallowOffsetOfMaxTimestamp, maxOffset);
if (maxTimestamp > this.maxTimestamp) {
this.maxTimestamp = maxTimestamp;
- this.offsetOfMaxTimestamp = offsetOfMaxTimestamp;
+ this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
}
this.maxOffset = Math.max(maxOffset, this.maxOffset);
this.messagesRetained += messagesRetained;
this.bytesRetained += bytesRetained;
}
- private void validateBatchMetadata(long maxTimestamp, long offsetOfMaxTimestamp, long maxOffset) {
- if (maxTimestamp != RecordBatch.NO_TIMESTAMP && offsetOfMaxTimestamp < 0)
- throw new IllegalArgumentException("offset undefined for maximum timestamp " + maxTimestamp);
+ private void validateBatchMetadata(long maxTimestamp, long shallowOffsetOfMaxTimestamp, long maxOffset) {
+ if (maxTimestamp != RecordBatch.NO_TIMESTAMP && shallowOffsetOfMaxTimestamp < 0)
+ throw new IllegalArgumentException("shallowOffset undefined for maximum timestamp " + maxTimestamp);
if (maxOffset < 0)
throw new IllegalArgumentException("maxOffset undefined");
}
@@ -459,8 +459,8 @@ public class MemoryRecords extends AbstractRecords {
return maxTimestamp;
}
- public long offsetOfMaxTimestamp() {
- return offsetOfMaxTimestamp;
+ public long shallowOffsetOfMaxTimestamp() {
+ return shallowOffsetOfMaxTimestamp;
}
}
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
index f183f7c2d19..88991328bb7 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
@@ -851,12 +851,12 @@ public class MemoryRecordsBuilder implements AutoCloseable {
public static class RecordsInfo {
public final long maxTimestamp;
- public final long offsetOfMaxTimestamp;
+ public final long shallowOffsetOfMaxTimestamp;
public RecordsInfo(long maxTimestamp,
- long offsetOfMaxTimestamp) {
+ long shallowOffsetOfMaxTimestamp) {
this.maxTimestamp = maxTimestamp;
- this.offsetOfMaxTimestamp = offsetOfMaxTimestamp;
+ this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
}
}
diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
index cf56b03b3f9..eb28ba47ef5 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
@@ -378,7 +378,7 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder.RecordsInfo info = builder.info();
assertEquals(logAppendTime, info.maxTimestamp);
// When logAppendTime is used, the first offset of the batch will be the offset of maxTimestamp
- assertEquals(0L, info.offsetOfMaxTimestamp);
+ assertEquals(0L, info.shallowOffsetOfMaxTimestamp);
for (RecordBatch batch : records.batches()) {
if (magic == MAGIC_VALUE_V0) {
@@ -414,9 +414,9 @@ public class MemoryRecordsBuilderTest {
if (magic == MAGIC_VALUE_V0)
// in MAGIC_VALUE_V0's case, we don't have timestamp info in records, so always return -1.
- assertEquals(-1L, info.offsetOfMaxTimestamp);
+ assertEquals(-1L, info.shallowOffsetOfMaxTimestamp);
else
- assertEquals(1L, info.offsetOfMaxTimestamp);
+ assertEquals(1L, info.shallowOffsetOfMaxTimestamp);
int i = 0;
long[] expectedTimestamps = new long[] {0L, 2L, 1L};
@@ -495,10 +495,10 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder.RecordsInfo info = builder.info();
if (magic == MAGIC_VALUE_V0) {
assertEquals(-1, info.maxTimestamp);
- assertEquals(-1L, info.offsetOfMaxTimestamp);
+ assertEquals(-1L, info.shallowOffsetOfMaxTimestamp);
} else {
assertEquals(2L, info.maxTimestamp);
- assertEquals(2L, info.offsetOfMaxTimestamp);
+ assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
}
long i = 0L;
diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
index 9e688fc3ab6..50821af841c 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
@@ -352,7 +352,7 @@ public class MemoryRecordsTest {
assertEquals(0, filterResult.messagesRetained());
assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained());
assertEquals(12, filterResult.maxTimestamp());
- assertEquals(baseOffset + 1, filterResult.offsetOfMaxTimestamp());
+ assertEquals(baseOffset + 1, filterResult.shallowOffsetOfMaxTimestamp());
// Verify filtered records
filtered.flip();
@@ -413,7 +413,7 @@ public class MemoryRecordsTest {
assertEquals(0, filterResult.messagesRetained());
assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained());
assertEquals(timestamp, filterResult.maxTimestamp());
- assertEquals(baseOffset, filterResult.offsetOfMaxTimestamp());
+ assertEquals(baseOffset, filterResult.shallowOffsetOfMaxTimestamp());
assertTrue(filterResult.outputBuffer().position() > 0);
// Verify filtered records
@@ -893,7 +893,7 @@ public class MemoryRecordsTest {
assertEquals(filtered.limit(), result.bytesRetained());
if (magic > RecordBatch.MAGIC_VALUE_V0) {
assertEquals(20L, result.maxTimestamp());
- assertEquals(4L, result.offsetOfMaxTimestamp());
+ assertEquals(4L, result.shallowOffsetOfMaxTimestamp());
}
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala
index dee0c52c4d2..92758d01144 100644
--- a/core/src/main/scala/kafka/log/LocalLog.scala
+++ b/core/src/main/scala/kafka/log/LocalLog.scala
@@ -406,8 +406,8 @@ class LocalLog(@volatile private var _dir: File,
}
}
- private[log] def append(lastOffset: Long, largestTimestamp: Long, offsetOfMaxTimestamp: Long, records: MemoryRecords): Unit = {
- segments.activeSegment.append(lastOffset, largestTimestamp, offsetOfMaxTimestamp, records)
+ private[log] def append(lastOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords): Unit = {
+ segments.activeSegment.append(lastOffset, largestTimestamp, shallowOffsetOfMaxTimestamp, records)
updateLogEndOffset(lastOffset + 1)
}
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
index b488bd5a8ca..0fa9a3b0488 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -812,7 +812,7 @@ private[log] class Cleaner(val id: Int,
val retained = MemoryRecords.readableRecords(outputBuffer)
// it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads
// after `Log.replaceSegments` (which acquires the lock) is called
- dest.append(result.maxOffset, result.maxTimestamp, result.offsetOfMaxTimestamp, retained)
+ dest.append(result.maxOffset, result.maxTimestamp, result.shallowOffsetOfMaxTimestamp, retained)
throttler.maybeThrottle(outputBuffer.limit())
}
diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala
index 63cd2b7ad23..85b655f16c2 100644
--- a/core/src/main/scala/kafka/log/UnifiedLog.scala
+++ b/core/src/main/scala/kafka/log/UnifiedLog.scala
@@ -817,7 +817,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
validRecords = validateAndOffsetAssignResult.validatedRecords
appendInfo.setMaxTimestamp(validateAndOffsetAssignResult.maxTimestampMs)
- appendInfo.setOffsetOfMaxTimestamp(validateAndOffsetAssignResult.offsetOfMaxTimestampMs)
+ appendInfo.setOffsetOfMaxTimestamp(validateAndOffsetAssignResult.shallowOffsetOfMaxTimestampMs)
appendInfo.setLastOffset(offset.value - 1)
appendInfo.setRecordValidationStats(validateAndOffsetAssignResult.recordValidationStats)
if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
diff --git a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
index 5362a1d5e35..e5e22e9dff9 100644
--- a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
@@ -68,7 +68,9 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
verifyListOffsets()
// test LogAppendTime case
- setUpForLogAppendTimeCase()
+ val props: Properties = new Properties()
+ props.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime")
+ createTopicWithConfig(topicNameWithCustomConfigs, props)
produceMessagesInOneBatch("gzip", topicNameWithCustomConfigs)
// In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch.
// So in this one batch test, it'll be the first offset 0
@@ -77,30 +79,9 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
@ValueSource(strings = Array("zk", "kraft"))
- def testThreeNonCompressedRecordsInOneBatch(quorum: String): Unit = {
- produceMessagesInOneBatch()
- verifyListOffsets()
-
- // test LogAppendTime case
- setUpForLogAppendTimeCase()
- produceMessagesInOneBatch(topic=topicNameWithCustomConfigs)
- // In LogAppendTime's case, if the timestamps are the same, we choose the offset of the first record
- // thus, the maxTimestampOffset should be the first record of the batch.
- // So in this one batch test, it'll be the first offset which is 0
- verifyListOffsets(topic = topicNameWithCustomConfigs, 0)
- }
-
- @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
- @ValueSource(strings = Array("zk", "kraft"))
- def testThreeNonCompressedRecordsInSeparateBatch(quorum: String): Unit = {
+ def testThreeRecordsInSeparateBatch(quorum: String): Unit = {
produceMessagesInSeparateBatch()
verifyListOffsets()
-
- // test LogAppendTime case
- setUpForLogAppendTimeCase()
- produceMessagesInSeparateBatch(topic = topicNameWithCustomConfigs)
- // In LogAppendTime's case, if the timestamp is different, it should be the last one
- verifyListOffsets(topic = topicNameWithCustomConfigs, 2)
}
// The message conversion test only run in ZK mode because KRaft mode doesn't support "inter.broker.protocol.version" < 3.0
@@ -112,7 +93,9 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
verifyListOffsets()
// test LogAppendTime case
- setUpForLogAppendTimeCase()
+ val props: Properties = new Properties()
+ props.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime")
+ createTopicWithConfig(topicNameWithCustomConfigs, props)
produceMessagesInOneBatch(topic = topicNameWithCustomConfigs)
// In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch.
// So in this one batch test, it'll be the first offset 0
@@ -128,7 +111,9 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
verifyListOffsets()
// test LogAppendTime case
- setUpForLogAppendTimeCase()
+ val props: Properties = new Properties()
+ props.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime")
+ createTopicWithConfig(topicNameWithCustomConfigs, props)
produceMessagesInSeparateBatch(topic = topicNameWithCustomConfigs)
// In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch.
// So in this separate batch test, it'll be the last offset 2
@@ -162,19 +147,15 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
verifyListOffsets()
// test LogAppendTime case
- setUpForLogAppendTimeCase()
+ val props: Properties = new Properties()
+ props.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime")
+ createTopicWithConfig(topicNameWithCustomConfigs, props)
produceMessagesInSeparateBatch("gzip", topicNameWithCustomConfigs)
// In LogAppendTime's case, the maxTimestampOffset should be the first message of the batch.
// So in this separate batch test, it'll be the last offset 2
verifyListOffsets(topic = topicNameWithCustomConfigs, 2)
}
- private def setUpForLogAppendTimeCase(): Unit = {
- val props: Properties = new Properties()
- props.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime")
- createTopicWithConfig(topicNameWithCustomConfigs, props)
- }
-
private def createOldMessageFormatBrokers(): Unit = {
setOldMessageFormat = true
recreateBrokers(reconfigure = true, startup = true)
diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
index bffd41156b3..29b5fd34f90 100644
--- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
@@ -100,7 +100,7 @@ class LocalLogTest {
initialOffset: Long = 0L): Unit = {
log.append(lastOffset = initialOffset + records.size - 1,
largestTimestamp = records.head.timestamp,
- offsetOfMaxTimestamp = initialOffset,
+ shallowOffsetOfMaxTimestamp = initialOffset,
records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
}
diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
index 8a5ae359630..11fff517b43 100644
--- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
@@ -85,10 +85,10 @@ class LogSegmentTest {
def testAppendForLogSegmentOffsetOverflowException(baseOffset: Long, largestOffset: Long): Unit = {
val seg = createSegment(baseOffset)
val currentTime = Time.SYSTEM.milliseconds()
- val offsetOfMaxTimestamp = largestOffset
+ val shallowOffsetOfMaxTimestamp = largestOffset
val memoryRecords = records(0, "hello")
assertThrows(classOf[LogSegmentOffsetOverflowException], () => {
- seg.append(largestOffset, currentTime, offsetOfMaxTimestamp, memoryRecords)
+ seg.append(largestOffset, currentTime, shallowOffsetOfMaxTimestamp, memoryRecords)
})
}
diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
index 53b385c62e8..ac6152b9b15 100644
--- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
@@ -173,9 +173,9 @@ class LogValidatorTest {
assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now")
assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed")
- // If it's LOG_APPEND_TIME, the offset will be the offset of the first record
- val expectedMaxTimestampOffset = 0
- assertEquals(expectedMaxTimestampOffset, validatedResults.offsetOfMaxTimestampMs,
+ // we index from last offset in version 2 instead of base offset
+ val expectedMaxTimestampOffset = if (magic >= RecordBatch.MAGIC_VALUE_V2) 2 else 0
+ assertEquals(expectedMaxTimestampOffset, validatedResults.shallowOffsetOfMaxTimestampMs,
s"The offset of max timestamp should be $expectedMaxTimestampOffset")
verifyRecordValidationStats(validatedResults.recordValidationStats, numConvertedRecords = 0, records,
compressed = false)
@@ -219,7 +219,7 @@ class LogValidatorTest {
"MessageSet should still valid")
assertEquals(now, validatedResults.maxTimestampMs,
s"Max timestamp should be $now")
- assertEquals(0, validatedResults.offsetOfMaxTimestampMs,
+ assertEquals(0, validatedResults.shallowOffsetOfMaxTimestampMs,
s"The offset of max timestamp should be 0 if logAppendTime is used")
assertTrue(validatedResults.messageSizeMaybeChanged,
"Message size may have been changed")
@@ -271,7 +271,7 @@ class LogValidatorTest {
"MessageSet should still valid")
assertEquals(now, validatedResults.maxTimestampMs,
s"Max timestamp should be $now")
- assertEquals(0, validatedResults.offsetOfMaxTimestampMs,
+ assertEquals(0, validatedResults.shallowOffsetOfMaxTimestampMs,
s"The offset of max timestamp should be 0 if logAppendTime is used")
assertFalse(validatedResults.messageSizeMaybeChanged,
"Message size should not have been changed")
@@ -404,8 +404,14 @@ class LogValidatorTest {
assertEquals(now + 1, validatingResults.maxTimestampMs,
s"Max timestamp should be ${now + 1}")
- val expectedOffsetOfMaxTimestamp = 1
- assertEquals(expectedOffsetOfMaxTimestamp, validatingResults.offsetOfMaxTimestampMs,
+ val expectedShallowOffsetOfMaxTimestamp = if (magic >= RecordVersion.V2.value) {
+ // v2 records are always batched, even when not compressed.
+ // the shallow offset of max timestamp is the last offset of the batch
+ recordList.size - 1
+ } else {
+ 1
+ }
+ assertEquals(expectedShallowOffsetOfMaxTimestamp, validatingResults.shallowOffsetOfMaxTimestampMs,
s"Offset of max timestamp should be 1")
assertFalse(validatingResults.messageSizeMaybeChanged,
@@ -480,7 +486,7 @@ class LogValidatorTest {
}
assertEquals(now + 1, validatingResults.maxTimestampMs,
s"Max timestamp should be ${now + 1}")
- assertEquals(1, validatingResults.offsetOfMaxTimestampMs,
+ assertEquals(1, validatingResults.shallowOffsetOfMaxTimestampMs,
"Offset of max timestamp should be 1")
assertTrue(validatingResults.messageSizeMaybeChanged,
"Message size should have been changed")
@@ -532,7 +538,7 @@ class LogValidatorTest {
}
assertEquals(validatedResults.maxTimestampMs, RecordBatch.NO_TIMESTAMP,
s"Max timestamp should be ${RecordBatch.NO_TIMESTAMP}")
- assertEquals(-1, validatedResults.offsetOfMaxTimestampMs,
+ assertEquals(-1, validatedResults.shallowOffsetOfMaxTimestampMs,
s"Offset of max timestamp should be -1")
assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed")
@@ -579,7 +585,7 @@ class LogValidatorTest {
assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence)
}
assertEquals(timestamp, validatedResults.maxTimestampMs)
- assertEquals(0, validatedResults.offsetOfMaxTimestampMs,
+ assertEquals(0, validatedResults.shallowOffsetOfMaxTimestampMs,
s"Offset of max timestamp should be 0 when multiple records having the same max timestamp.")
assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed")
@@ -651,8 +657,8 @@ class LogValidatorTest {
}
assertEquals(now + 1, validatedResults.maxTimestampMs, s"Max timestamp should be ${now + 1}")
- val expectedOffsetOfMaxTimestamp = 1
- assertEquals(expectedOffsetOfMaxTimestamp, validatedResults.offsetOfMaxTimestampMs,
+ val expectedShallowOffsetOfMaxTimestamp = 1
+ assertEquals(expectedShallowOffsetOfMaxTimestamp, validatedResults.shallowOffsetOfMaxTimestampMs,
s"Offset of max timestamp should be 1")
assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed")
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java
index d47a4a71824..cdb03e72b34 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java
@@ -238,17 +238,17 @@ public class LogSegment implements Closeable {
*
* @param largestOffset The last offset in the message set
* @param largestTimestampMs The largest timestamp in the message set.
- * @param offsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append.
+ * @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages 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 offsetOfMaxTimestamp,
+ 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, offsetOfMaxTimestamp);
+ LOGGER.trace("Inserting {} bytes at end offset {} at position {} with largest timestamp {} at shallow offset {}",
+ records.sizeInBytes(), largestOffset, log.sizeInBytes(), largestTimestampMs, shallowOffsetOfMaxTimestamp);
int physicalPosition = log.sizeInBytes();
if (physicalPosition == 0)
rollingBasedTimestamp = OptionalLong.of(largestTimestampMs);
@@ -260,7 +260,7 @@ public class LogSegment implements Closeable {
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, offsetOfMaxTimestamp);
+ maxTimestampAndOffsetSoFar = new TimestampOffset(largestTimestampMs, shallowOffsetOfMaxTimestamp);
}
// append an entry to the index (if needed)
if (bytesSinceLastIndexEntry > indexIntervalBytes) {
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java
index 9aa1e06633b..0cf9cd1c60f 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java
@@ -68,17 +68,17 @@ public class LogValidator {
public final long logAppendTimeMs;
public final MemoryRecords validatedRecords;
public final long maxTimestampMs;
- public final long offsetOfMaxTimestampMs;
+ public final long shallowOffsetOfMaxTimestampMs;
public final boolean messageSizeMaybeChanged;
public final RecordValidationStats recordValidationStats;
public ValidationResult(long logAppendTimeMs, MemoryRecords validatedRecords, long maxTimestampMs,
- long offsetOfMaxTimestampMs, boolean messageSizeMaybeChanged,
+ long shallowOffsetOfMaxTimestampMs, boolean messageSizeMaybeChanged,
RecordValidationStats recordValidationStats) {
this.logAppendTimeMs = logAppendTimeMs;
this.validatedRecords = validatedRecords;
this.maxTimestampMs = maxTimestampMs;
- this.offsetOfMaxTimestampMs = offsetOfMaxTimestampMs;
+ this.shallowOffsetOfMaxTimestampMs = shallowOffsetOfMaxTimestampMs;
this.messageSizeMaybeChanged = messageSizeMaybeChanged;
this.recordValidationStats = recordValidationStats;
}
@@ -149,7 +149,7 @@ public class LogValidator {
* avoid expensive re-compression.
*
* Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
- * of the message with the max timestamp and a boolean indicating whether the message sizes may have changed.
+ * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
*/
public ValidationResult validateMessagesAndAssignOffsets(PrimitiveRef.LongRef offsetCounter,
MetricsRecorder metricsRecorder,
@@ -232,7 +232,7 @@ public class LogValidator {
now,
convertedRecords,
info.maxTimestamp,
- info.offsetOfMaxTimestamp,
+ info.shallowOffsetOfMaxTimestamp,
true,
recordValidationStats);
}
@@ -296,6 +296,10 @@ public class LogValidator {
offsetOfMaxTimestamp = initialOffset;
}
+ if (toMagic >= RecordBatch.MAGIC_VALUE_V2) {
+ offsetOfMaxTimestamp = offsetCounter.value - 1;
+ }
+
return new ValidationResult(
now,
records,
@@ -476,7 +480,7 @@ public class LogValidator {
logAppendTime,
records,
info.maxTimestamp,
- info.offsetOfMaxTimestamp,
+ info.shallowOffsetOfMaxTimestamp,
true,
recordValidationStats);
}