From e8232edd24c6dd31bc58d29967d552f3dc6cf831 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 21 Dec 2022 16:57:02 -0800 Subject: [PATCH] KAFKA-14477: Move LogValidator and related to storage module (#13012) Also improved `LogValidatorTest` to cover a bug that was originally only caught by `LogAppendTimeTest`. For broader context on this change, please check: * KAFKA-14470: Move log layer to storage module Reviewers: Jun Rao --- build.gradle | 1 + checkstyle/suppressions.xml | 10 +- .../kafka/common/utils/PrimitiveRef.java | 23 + .../kafka/common/utils/PrimitiveRefTest.java | 43 + .../main/scala/kafka/cluster/Partition.scala | 1 + .../common/RecordValidationException.scala | 28 - .../group/GroupMetadataManager.scala | 7 +- .../transaction/TransactionStateManager.scala | 8 +- .../src/main/scala/kafka/log/LogSegment.scala | 4 +- .../main/scala/kafka/log/LogValidator.scala | 592 -------- .../kafka/log/ProducerStateManager.scala | 8 +- .../src/main/scala/kafka/log/UnifiedLog.scala | 79 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 5 +- .../main/scala/kafka/server/KafkaApis.scala | 6 +- .../scala/kafka/server/ReplicaManager.scala | 4 +- .../server/LocalLeaderEndPointTest.scala | 6 +- .../kafka/cluster/PartitionLockTest.scala | 3 +- .../unit/kafka/cluster/PartitionTest.scala | 13 +- .../AbstractCoordinatorConcurrencyTest.scala | 4 +- .../group/GroupCoordinatorTest.scala | 10 +- .../group/GroupMetadataManagerTest.scala | 45 +- .../TransactionStateManagerTest.scala | 14 +- .../kafka/log/LogCleanerManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 66 +- .../scala/unit/kafka/log/LogTestUtils.scala | 4 +- .../unit/kafka/log/LogValidatorTest.scala | 1246 ++++++++--------- .../kafka/log/ProducerStateManagerTest.scala | 40 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 22 +- .../unit/kafka/server/KafkaApisTest.scala | 10 +- .../ReplicaManagerConcurrencyTest.scala | 5 +- .../kafka/server/ReplicaManagerTest.scala | 17 +- .../kafka/tools/DumpLogSegmentsTest.scala | 6 +- gradle/spotbugs-exclude.xml | 7 + .../jmh/record/BaseRecordBatchBenchmark.java | 5 +- ...pressedRecordBatchValidationBenchmark.java | 21 +- ...pressedRecordBatchValidationBenchmark.java | 17 +- .../server/log/internals/AppendOrigin.java | 45 + .../server/log/internals/LogValidator.java | 631 +++++++++ .../internals/RecordValidationException.java | 41 + 39 files changed, 1653 insertions(+), 1448 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java delete mode 100644 core/src/main/scala/kafka/common/RecordValidationException.scala delete mode 100644 core/src/main/scala/kafka/log/LogValidator.scala create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java diff --git a/build.gradle b/build.gradle index 81ab8e8c211..776bbbcc699 100644 --- a/build.gradle +++ b/build.gradle @@ -2413,6 +2413,7 @@ project(':jmh-benchmarks') { implementation project(':server-common') implementation project(':clients') implementation project(':metadata') + implementation project(':storage') implementation project(':streams') implementation project(':core') implementation project(':clients').sourceSets.test.output diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index cea6a193790..f49c3e8edb6 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -318,9 +318,13 @@ - - + + + + GroupMetadataKeyData} -import kafka.log.AppendOrigin import kafka.metrics.KafkaMetricsGroup import kafka.server.{FetchLogEnd, ReplicaManager, RequestLocal} import kafka.utils.CoreUtils.inLock @@ -48,6 +46,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, MessageFormatter, TopicPartition} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_1_IV0, IBP_2_1_IV0, IBP_2_1_IV1, IBP_2_3_IV0} +import org.apache.kafka.server.log.internals.AppendOrigin import scala.collection._ import scala.collection.mutable.ArrayBuffer @@ -331,7 +330,7 @@ class GroupMetadataManager(brokerId: Int, timeout = config.offsetCommitTimeoutMs.toLong, requiredAcks = config.offsetCommitRequiredAcks, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, entriesPerPartition = records, delayedProduceLock = Some(group.lock), responseCallback = callback, @@ -890,7 +889,7 @@ class GroupMetadataManager(brokerId: Int, // do not need to require acks since even if the tombstone is lost, // it will be appended again in the next purge cycle val records = MemoryRecords.withRecords(magicValue, 0L, compressionType, timestampType, tombstones.toArray: _*) - partition.appendRecordsToLeader(records, origin = AppendOrigin.Coordinator, requiredAcks = 0, + partition.appendRecordsToLeader(records, origin = AppendOrigin.COORDINATOR, requiredAcks = 0, requestLocal = requestLocal) offsetsRemoved += removedOffsets.size diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index ad721ccc380..9073d491abc 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -16,12 +16,13 @@ */ package kafka.coordinator.transaction +import kafka.log.LogConfig + import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.log.{AppendOrigin, LogConfig} import kafka.server.{Defaults, FetchLogEnd, ReplicaManager, RequestLocal} import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.{Logging, Pool, Scheduler} @@ -36,6 +37,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.server.record.BrokerCompressionType import scala.jdk.CollectionConverters._ @@ -282,7 +284,7 @@ class TransactionStateManager(brokerId: Int, config.requestTimeoutMs, TransactionLog.EnforcedRequiredAcks, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, entriesPerPartition = Map(transactionPartition -> tombstoneRecords), removeFromCacheCallback, requestLocal = RequestLocal.NoCaching) @@ -761,7 +763,7 @@ class TransactionStateManager(brokerId: Int, newMetadata.txnTimeoutMs.toLong, TransactionLog.EnforcedRequiredAcks, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, recordsPerPartition, updateCacheCallback, requestLocal = requestLocal) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index ae9a6cbabc8..93677bee653 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.FileRecords.{LogOffsetPosition, TimestampAndOffset} import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time} -import org.apache.kafka.server.log.internals.{AbortedTxn, CompletedTxn, LazyIndex, OffsetIndex, OffsetPosition, TimeIndex, TimestampOffset, TransactionIndex, TxnIndexSearchResult} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LazyIndex, OffsetIndex, OffsetPosition, TimeIndex, TimestampOffset, TransactionIndex, TxnIndexSearchResult} import java.util.Optional import scala.jdk.CollectionConverters._ @@ -248,7 +248,7 @@ class LogSegment private[log] (val log: FileRecords, private def updateProducerState(producerStateManager: ProducerStateManager, batch: RecordBatch): Unit = { if (batch.hasProducerId) { val producerId = batch.producerId - val appendInfo = producerStateManager.prepareUpdate(producerId, origin = AppendOrigin.Replication) + val appendInfo = producerStateManager.prepareUpdate(producerId, origin = AppendOrigin.REPLICATION) val maybeCompletedTxn = appendInfo.append(batch, firstOffsetMetadataOpt = None) producerStateManager.update(appendInfo) maybeCompletedTxn.foreach { completedTxn => diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala deleted file mode 100644 index e36aceab01d..00000000000 --- a/core/src/main/scala/kafka/log/LogValidator.scala +++ /dev/null @@ -1,592 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.log - -import java.nio.ByteBuffer - -import kafka.common.{LongRef, RecordValidationException} -import kafka.server.{BrokerTopicStats, RequestLocal} -import kafka.utils.Logging -import org.apache.kafka.common.errors.{CorruptRecordException, InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException} -import org.apache.kafka.common.record.{AbstractRecords, CompressionType, MemoryRecords, Record, RecordBatch, RecordConversionStats, TimestampType} -import org.apache.kafka.common.InvalidRecordException -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.ProduceResponse.RecordError -import org.apache.kafka.common.utils.Time -import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.common.MetadataVersion.IBP_2_1_IV0 - -import scala.collection.{Seq, mutable} -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.ArrayBuffer - -/** - * The source of an append to the log. This is used when determining required validations. - */ -private[kafka] sealed trait AppendOrigin -private[kafka] object AppendOrigin { - - /** - * The log append came through replication from the leader. This typically implies minimal validation. - * Particularly, we do not decompress record batches in order to validate records individually. - */ - case object Replication extends AppendOrigin - - /** - * The log append came from either the group coordinator or the transaction coordinator. We validate - * producer epochs for normal log entries (specifically offset commits from the group coordinator) and - * we validate coordinate end transaction markers from the transaction coordinator. - */ - case object Coordinator extends AppendOrigin - - /** - * The log append came from the client, which implies full validation. - */ - case object Client extends AppendOrigin - - /** - * The log append come from the raft leader, which implies the offsets has been assigned - */ - case object RaftLeader extends AppendOrigin -} - -private[log] object LogValidator extends Logging { - - /** - * Update the offsets for this message set and do further validation on messages including: - * 1. Messages for compacted topics must have keys - * 2. When magic value >= 1, inner messages of a compressed message set must have monotonically increasing offsets - * starting from 0. - * 3. When magic value >= 1, validate and maybe overwrite timestamps of messages. - * 4. Declared count of records in DefaultRecordBatch must match number of valid records contained therein. - * - * This method will convert messages as necessary to the topic's configured message format version. If no format - * conversion or value overwriting is required for messages, this method will perform in-place operations to - * avoid expensive re-compression. - * - * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset - * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed. - */ - private[log] def validateMessagesAndAssignOffsets(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - time: Time, - now: Long, - sourceCompression: CompressionType, - targetCompression: CompressionType, - compactedTopic: Boolean, - magic: Byte, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - interBrokerProtocolVersion: MetadataVersion, - brokerTopicStats: BrokerTopicStats, - requestLocal: RequestLocal): ValidationAndOffsetAssignResult = { - if (sourceCompression == CompressionType.NONE && targetCompression == CompressionType.NONE) { - // check the magic value - if (!records.hasMatchingMagic(magic)) - convertAndAssignOffsetsNonCompressed(records, topicPartition, offsetCounter, compactedTopic, time, now, timestampType, - timestampDiffMaxMs, magic, partitionLeaderEpoch, origin, brokerTopicStats) - else - // Do in-place validation, offset assignment and maybe set timestamp - assignOffsetsNonCompressed(records, topicPartition, offsetCounter, now, compactedTopic, timestampType, timestampDiffMaxMs, - partitionLeaderEpoch, origin, magic, brokerTopicStats) - } else { - validateMessagesAndAssignOffsetsCompressed(records, topicPartition, offsetCounter, time, now, sourceCompression, - targetCompression, compactedTopic, magic, timestampType, timestampDiffMaxMs, partitionLeaderEpoch, origin, - interBrokerProtocolVersion, brokerTopicStats, requestLocal) - } - } - - private def getFirstBatchAndMaybeValidateNoMoreBatches(records: MemoryRecords, sourceCompression: CompressionType): RecordBatch = { - val batchIterator = records.batches.iterator - - if (!batchIterator.hasNext) { - throw new InvalidRecordException("Record batch has no batches at all") - } - - val batch = batchIterator.next() - - // if the format is v2 and beyond, or if the messages are compressed, we should check there's only one batch. - if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2 || sourceCompression != CompressionType.NONE) { - if (batchIterator.hasNext) { - throw new InvalidRecordException("Compressed outer record has more than one batch") - } - } - - batch - } - - private def validateBatch(topicPartition: TopicPartition, - firstBatch: RecordBatch, - batch: RecordBatch, - origin: AppendOrigin, - toMagic: Byte, - brokerTopicStats: BrokerTopicStats): Unit = { - // batch magic byte should have the same magic as the first batch - if (firstBatch.magic() != batch.magic()) { - brokerTopicStats.allTopicsStats.invalidMagicNumberRecordsPerSec.mark() - throw new InvalidRecordException(s"Batch magic ${batch.magic()} is not the same as the first batch'es magic byte ${firstBatch.magic()} in topic partition $topicPartition.") - } - - if (origin == AppendOrigin.Client) { - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) { - val countFromOffsets = batch.lastOffset - batch.baseOffset + 1 - if (countFromOffsets <= 0) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Batch has an invalid offset range: [${batch.baseOffset}, ${batch.lastOffset}] in topic partition $topicPartition.") - } - - // v2 and above messages always have a non-null count - val count = batch.countOrNull - if (count <= 0) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Invalid reported count for record batch: $count in topic partition $topicPartition.") - } - - if (countFromOffsets != batch.countOrNull) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Inconsistent batch offset range [${batch.baseOffset}, ${batch.lastOffset}] " + - s"and count of records $count in topic partition $topicPartition.") - } - } - - if (batch.isControlBatch) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Clients are not allowed to write control records in topic partition $topicPartition.") - } - - if (batch.hasProducerId && batch.baseSequence < 0) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Invalid sequence number ${batch.baseSequence} in record batch " + - s"with producerId ${batch.producerId} in topic partition $topicPartition.") - } - } - - if (batch.isTransactional && toMagic < RecordBatch.MAGIC_VALUE_V2) - throw new UnsupportedForMessageFormatException(s"Transactional records cannot be used with magic version $toMagic") - - if (batch.hasProducerId && toMagic < RecordBatch.MAGIC_VALUE_V2) - throw new UnsupportedForMessageFormatException(s"Idempotent records cannot be used with magic version $toMagic") - } - - private def validateRecord(batch: RecordBatch, topicPartition: TopicPartition, record: Record, batchIndex: Int, now: Long, - timestampType: TimestampType, timestampDiffMaxMs: Long, compactedTopic: Boolean, - brokerTopicStats: BrokerTopicStats): Option[ApiRecordError] = { - if (!record.hasMagic(batch.magic)) { - brokerTopicStats.allTopicsStats.invalidMagicNumberRecordsPerSec.mark() - return Some(ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, - s"Record $record's magic does not match outer magic ${batch.magic} in topic partition $topicPartition."))) - } - - // verify the record-level CRC only if this is one of the deep entries of a compressed message - // set for magic v0 and v1. For non-compressed messages, there is no inner record for magic v0 and v1, - // so we depend on the batch-level CRC check in Log.analyzeAndValidateRecords(). For magic v2 and above, - // there is no record-level CRC to check. - if (batch.magic <= RecordBatch.MAGIC_VALUE_V1 && batch.isCompressed) { - try { - record.ensureValid() - } catch { - case e: InvalidRecordException => - brokerTopicStats.allTopicsStats.invalidMessageCrcRecordsPerSec.mark() - throw new CorruptRecordException(e.getMessage + s" in topic partition $topicPartition.") - } - } - - validateKey(record, batchIndex, topicPartition, compactedTopic, brokerTopicStats).orElse { - validateTimestamp(batch, record, batchIndex, now, timestampType, timestampDiffMaxMs) - } - } - - private def convertAndAssignOffsetsNonCompressed(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - compactedTopic: Boolean, - time: Time, - now: Long, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - toMagicValue: Byte, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - brokerTopicStats: BrokerTopicStats): ValidationAndOffsetAssignResult = { - val startNanos = time.nanoseconds - val sizeInBytesAfterConversion = AbstractRecords.estimateSizeInBytes(toMagicValue, offsetCounter.value, - CompressionType.NONE, records.records) - - val (producerId, producerEpoch, sequence, isTransactional) = { - val first = records.batches.asScala.head - (first.producerId, first.producerEpoch, first.baseSequence, first.isTransactional) - } - - // The current implementation of BufferSupplier is naive and works best when the buffer size - // cardinality is low, so don't use it here - val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion) - val builder = MemoryRecords.builder(newBuffer, toMagicValue, CompressionType.NONE, timestampType, - offsetCounter.value, now, producerId, producerEpoch, sequence, isTransactional, partitionLeaderEpoch) - - val firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE) - - records.batches.forEach { batch => - validateBatch(topicPartition, firstBatch, batch, origin, toMagicValue, brokerTopicStats) - - val recordErrors = new ArrayBuffer[ApiRecordError](0) - for ((record, batchIndex) <- batch.asScala.view.zipWithIndex) { - validateRecord(batch, topicPartition, record, batchIndex, now, timestampType, - timestampDiffMaxMs, compactedTopic, brokerTopicStats).foreach(recordError => recordErrors += recordError) - // we fail the batch if any record fails, so we stop appending if any record fails - if (recordErrors.isEmpty) - builder.appendWithOffset(offsetCounter.getAndIncrement(), record) - } - - processRecordErrors(recordErrors) - } - - val convertedRecords = builder.build() - - val info = builder.info - val recordConversionStats = new RecordConversionStats(builder.uncompressedBytesWritten, - builder.numRecords, time.nanoseconds - startNanos) - ValidationAndOffsetAssignResult( - validatedRecords = convertedRecords, - maxTimestamp = info.maxTimestamp, - shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp, - messageSizeMaybeChanged = true, - recordConversionStats = recordConversionStats) - } - - def assignOffsetsNonCompressed(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - now: Long, - compactedTopic: Boolean, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - magic: Byte, - brokerTopicStats: BrokerTopicStats): ValidationAndOffsetAssignResult = { - var maxTimestamp = RecordBatch.NO_TIMESTAMP - var offsetOfMaxTimestamp = -1L - val initialOffset = offsetCounter.value - - val firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE) - - records.batches.forEach { batch => - validateBatch(topicPartition, firstBatch, batch, origin, magic, brokerTopicStats) - - var maxBatchTimestamp = RecordBatch.NO_TIMESTAMP - var offsetOfMaxBatchTimestamp = -1L - - val recordErrors = new ArrayBuffer[ApiRecordError](0) - // This is a hot path and we want to avoid any unnecessary allocations. - // That said, there is no benefit in using `skipKeyValueIterator` for the uncompressed - // case since we don't do key/value copies in this path (we just slice the ByteBuffer) - var batchIndex = 0 - batch.forEach { record => - validateRecord(batch, topicPartition, record, batchIndex, now, timestampType, - timestampDiffMaxMs, compactedTopic, brokerTopicStats).foreach(recordError => recordErrors += recordError) - - val offset = offsetCounter.getAndIncrement() - if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && record.timestamp > maxBatchTimestamp) { - maxBatchTimestamp = record.timestamp - offsetOfMaxBatchTimestamp = offset - } - batchIndex += 1 - } - - processRecordErrors(recordErrors) - - if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && maxBatchTimestamp > maxTimestamp) { - maxTimestamp = maxBatchTimestamp - offsetOfMaxTimestamp = offsetOfMaxBatchTimestamp - } - - batch.setLastOffset(offsetCounter.value - 1) - - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) - batch.setPartitionLeaderEpoch(partitionLeaderEpoch) - - if (batch.magic > RecordBatch.MAGIC_VALUE_V0) { - if (timestampType == TimestampType.LOG_APPEND_TIME) - batch.setMaxTimestamp(TimestampType.LOG_APPEND_TIME, now) - else - batch.setMaxTimestamp(timestampType, maxBatchTimestamp) - } - } - - if (timestampType == TimestampType.LOG_APPEND_TIME) { - maxTimestamp = now - if (magic >= RecordBatch.MAGIC_VALUE_V2) - offsetOfMaxTimestamp = offsetCounter.value - 1 - else - offsetOfMaxTimestamp = initialOffset - } - - ValidationAndOffsetAssignResult( - validatedRecords = records, - maxTimestamp = maxTimestamp, - shallowOffsetOfMaxTimestamp = offsetOfMaxTimestamp, - messageSizeMaybeChanged = false, - recordConversionStats = RecordConversionStats.EMPTY) - } - - /** - * We cannot do in place assignment in one of the following situations: - * 1. Source and target compression codec are different - * 2. When the target magic is not equal to batches' magic, meaning format conversion is needed. - * 3. When the target magic is equal to V0, meaning absolute offsets need to be re-assigned. - */ - def validateMessagesAndAssignOffsetsCompressed(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - time: Time, - now: Long, - sourceCompression: CompressionType, - targetCompression: CompressionType, - compactedTopic: Boolean, - toMagic: Byte, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - interBrokerProtocolVersion: MetadataVersion, - brokerTopicStats: BrokerTopicStats, - requestLocal: RequestLocal): ValidationAndOffsetAssignResult = { - - if (targetCompression == CompressionType.ZSTD && interBrokerProtocolVersion.isLessThan(IBP_2_1_IV0)) - throw new UnsupportedCompressionTypeException("Produce requests to inter.broker.protocol.version < 2.1 broker " + - "are not allowed to use ZStandard compression") - - def validateRecordCompression(batchIndex: Int, record: Record): Option[ApiRecordError] = { - if (sourceCompression != CompressionType.NONE && record.isCompressed) - Some(ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, - s"Compressed outer record should not have an inner record with a compression attribute set: $record"))) - else None - } - - // No in place assignment situation 1 - var inPlaceAssignment = sourceCompression == targetCompression - - var maxTimestamp = RecordBatch.NO_TIMESTAMP - val expectedInnerOffset = new LongRef(0) - val validatedRecords = new mutable.ArrayBuffer[Record] - - var uncompressedSizeInBytes = 0 - - // Assume there's only one batch with compressed memory records; otherwise, return InvalidRecordException - // One exception though is that with format smaller than v2, if sourceCompression is noCompression, then each batch is actually - // a single record so we'd need to special handle it by creating a single wrapper batch that includes all the records - val firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, sourceCompression) - - // No in place assignment situation 2 and 3: we only need to check for the first batch because: - // 1. For most cases (compressed records, v2, for example), there's only one batch anyways. - // 2. For cases that there may be multiple batches, all batches' magic should be the same. - if (firstBatch.magic != toMagic || toMagic == RecordBatch.MAGIC_VALUE_V0) - inPlaceAssignment = false - - // Do not compress control records unless they are written compressed - if (sourceCompression == CompressionType.NONE && firstBatch.isControlBatch) - inPlaceAssignment = true - - records.batches.forEach { batch => - validateBatch(topicPartition, firstBatch, batch, origin, toMagic, brokerTopicStats) - uncompressedSizeInBytes += AbstractRecords.recordBatchHeaderSizeInBytes(toMagic, batch.compressionType()) - - // if we are on version 2 and beyond, and we know we are going for in place assignment, - // then we can optimize the iterator to skip key / value / headers since they would not be used at all - val recordsIterator = if (inPlaceAssignment && firstBatch.magic >= RecordBatch.MAGIC_VALUE_V2) - batch.skipKeyValueIterator(requestLocal.bufferSupplier) - else - batch.streamingIterator(requestLocal.bufferSupplier) - - try { - val recordErrors = new ArrayBuffer[ApiRecordError](0) - // this is a hot path and we want to avoid any unnecessary allocations. - var batchIndex = 0 - recordsIterator.forEachRemaining { record => - val expectedOffset = expectedInnerOffset.getAndIncrement() - val recordError = validateRecordCompression(batchIndex, record).orElse { - validateRecord(batch, topicPartition, record, batchIndex, now, - timestampType, timestampDiffMaxMs, compactedTopic, brokerTopicStats).orElse { - if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && toMagic > RecordBatch.MAGIC_VALUE_V0) { - if (record.timestamp > maxTimestamp) - maxTimestamp = record.timestamp - - // Some older clients do not implement the V1 internal offsets correctly. - // Historically the broker handled this by rewriting the batches rather - // than rejecting the request. We must continue this handling here to avoid - // breaking these clients. - if (record.offset != expectedOffset) - inPlaceAssignment = false - } - None - } - } - - recordError match { - case Some(e) => recordErrors += e - case None => - uncompressedSizeInBytes += record.sizeInBytes() - validatedRecords += record - } - batchIndex += 1 - } - processRecordErrors(recordErrors) - } finally { - recordsIterator.close() - } - } - - if (!inPlaceAssignment) { - val (producerId, producerEpoch, sequence, isTransactional) = { - // note that we only reassign offsets for requests coming straight from a producer. For records with magic V2, - // there should be exactly one RecordBatch per request, so the following is all we need to do. For Records - // with older magic versions, there will never be a producer id, etc. - val first = records.batches.asScala.head - (first.producerId, first.producerEpoch, first.baseSequence, first.isTransactional) - } - buildRecordsAndAssignOffsets(toMagic, offsetCounter, time, timestampType, targetCompression, now, validatedRecords, - producerId, producerEpoch, sequence, isTransactional, partitionLeaderEpoch, uncompressedSizeInBytes) - } else { - // we can update the batch only and write the compressed payload as is; - // again we assume only one record batch within the compressed set - val batch = records.batches.iterator.next() - val lastOffset = offsetCounter.addAndGet(validatedRecords.size) - 1 - - batch.setLastOffset(lastOffset) - - if (timestampType == TimestampType.LOG_APPEND_TIME) - maxTimestamp = now - - if (toMagic >= RecordBatch.MAGIC_VALUE_V1) - batch.setMaxTimestamp(timestampType, maxTimestamp) - - if (toMagic >= RecordBatch.MAGIC_VALUE_V2) - batch.setPartitionLeaderEpoch(partitionLeaderEpoch) - - val recordConversionStats = new RecordConversionStats(uncompressedSizeInBytes, 0, 0) - ValidationAndOffsetAssignResult(validatedRecords = records, - maxTimestamp = maxTimestamp, - shallowOffsetOfMaxTimestamp = lastOffset, - messageSizeMaybeChanged = false, - recordConversionStats = recordConversionStats) - } - } - - private def buildRecordsAndAssignOffsets(magic: Byte, - offsetCounter: LongRef, - time: Time, - timestampType: TimestampType, - compressionType: CompressionType, - logAppendTime: Long, - validatedRecords: Seq[Record], - producerId: Long, - producerEpoch: Short, - baseSequence: Int, - isTransactional: Boolean, - partitionLeaderEpoch: Int, - uncompressedSizeInBytes: Int): ValidationAndOffsetAssignResult = { - val startNanos = time.nanoseconds - val estimatedSize = AbstractRecords.estimateSizeInBytes(magic, offsetCounter.value, compressionType, - validatedRecords.asJava) - // The current implementation of BufferSupplier is naive and works best when the buffer size - // cardinality is low, so don't use it here - val buffer = ByteBuffer.allocate(estimatedSize) - val builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType, offsetCounter.value, - logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) - - validatedRecords.foreach { record => - builder.appendWithOffset(offsetCounter.getAndIncrement(), record) - } - - val records = builder.build() - - val info = builder.info - - // This is not strictly correct, it represents the number of records where in-place assignment is not possible - // instead of the number of records that were converted. It will over-count cases where the source and target are - // message format V0 or if the inner offsets are not consecutive. This is OK since the impact is the same: we have - // to rebuild the records (including recompression if enabled). - val conversionCount = builder.numRecords - val recordConversionStats = new RecordConversionStats(uncompressedSizeInBytes + builder.uncompressedBytesWritten, - conversionCount, time.nanoseconds - startNanos) - - ValidationAndOffsetAssignResult( - validatedRecords = records, - maxTimestamp = info.maxTimestamp, - shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp, - messageSizeMaybeChanged = true, - recordConversionStats = recordConversionStats) - } - - private def validateKey(record: Record, - batchIndex: Int, - topicPartition: TopicPartition, - compactedTopic: Boolean, - brokerTopicStats: BrokerTopicStats): Option[ApiRecordError] = { - if (compactedTopic && !record.hasKey) { - brokerTopicStats.allTopicsStats.noKeyCompactedTopicRecordsPerSec.mark() - Some(ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, - s"Compacted topic cannot accept message without key in topic partition $topicPartition."))) - } else None - } - - private def validateTimestamp(batch: RecordBatch, - record: Record, - batchIndex: Int, - now: Long, - timestampType: TimestampType, - timestampDiffMaxMs: Long): Option[ApiRecordError] = { - if (timestampType == TimestampType.CREATE_TIME - && record.timestamp != RecordBatch.NO_TIMESTAMP - && math.abs(record.timestamp - now) > timestampDiffMaxMs) - Some(ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, - s"Timestamp ${record.timestamp} of message with offset ${record.offset} is " + - s"out of range. The timestamp should be within [${now - timestampDiffMaxMs}, " + - s"${now + timestampDiffMaxMs}]"))) - else if (batch.timestampType == TimestampType.LOG_APPEND_TIME) - Some(ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, - s"Invalid timestamp type in message $record. Producer should not set timestamp " + - "type to LogAppendTime."))) - else None - } - - private def processRecordErrors(recordErrors: Seq[ApiRecordError]): Unit = { - if (recordErrors.nonEmpty) { - val errors = recordErrors.map(_.recordError) - if (recordErrors.exists(_.apiError == Errors.INVALID_TIMESTAMP)) { - throw new RecordValidationException(new InvalidTimestampException( - "One or more records have been rejected due to invalid timestamp"), errors) - } else { - throw new RecordValidationException(new InvalidRecordException( - "One or more records have been rejected due to " + errors.size + " record errors " + - "in total, and only showing the first three errors at most: " + errors.asJava.subList(0, math.min(errors.size, 3))), errors) - } - } - } - - case class ValidationAndOffsetAssignResult(validatedRecords: MemoryRecords, - maxTimestamp: Long, - shallowOffsetOfMaxTimestamp: Long, - messageSizeMaybeChanged: Boolean, - recordConversionStats: RecordConversionStats) - - private case class ApiRecordError(apiError: Errors, recordError: RecordError) -} diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala index 7307bed0efa..1b9c4593d02 100644 --- a/core/src/main/scala/kafka/log/ProducerStateManager.scala +++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.protocol.types._ import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, EndTransactionMarker, RecordBatch} import org.apache.kafka.common.utils.{ByteUtils, Crc32C, Time, Utils} -import org.apache.kafka.server.log.internals.CompletedTxn +import org.apache.kafka.server.log.internals.{AppendOrigin, CompletedTxn} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -195,7 +195,7 @@ private[log] class ProducerAppendInfo(val topicPartition: TopicPartition, private def maybeValidateDataBatch(producerEpoch: Short, firstSeq: Int, offset: Long): Unit = { checkProducerEpoch(producerEpoch, offset) - if (origin == AppendOrigin.Client) { + if (origin == AppendOrigin.CLIENT) { checkSequence(producerEpoch, firstSeq, offset) } } @@ -205,7 +205,7 @@ private[log] class ProducerAppendInfo(val topicPartition: TopicPartition, val message = s"Epoch of producer $producerId at offset $offset in $topicPartition is $producerEpoch, " + s"which is smaller than the last seen epoch ${updatedEntry.producerEpoch}" - if (origin == AppendOrigin.Replication) { + if (origin == AppendOrigin.REPLICATION) { warn(message) } else { // Starting from 2.7, we replaced ProducerFenced error with InvalidProducerEpoch in the @@ -294,7 +294,7 @@ private[log] class ProducerAppendInfo(val topicPartition: TopicPartition, private def checkCoordinatorEpoch(endTxnMarker: EndTransactionMarker, offset: Long): Unit = { if (updatedEntry.coordinatorEpoch > endTxnMarker.coordinatorEpoch) { - if (origin == AppendOrigin.Replication) { + if (origin == AppendOrigin.REPLICATION) { info(s"Detected invalid coordinator epoch for producerId $producerId at " + s"offset $offset in partition $topicPartition: ${endTxnMarker.coordinatorEpoch} " + s"is older than previously known coordinator epoch ${updatedEntry.coordinatorEpoch}") diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index e7a27c35f85..8c2e4f74626 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -23,13 +23,12 @@ import java.io.{File, IOException} import java.nio.file.Files import java.util.Optional import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, TimeUnit} -import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} -import kafka.log.AppendOrigin.RaftLeader +import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.remote.RemoteLogManager import kafka.metrics.KafkaMetricsGroup -import kafka.server._ import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.LeaderEpochFileCache +import kafka.server.{BrokerTopicMetrics, BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, LogDirFailureChannel, LogOffsetMetadata, OffsetAndEpoch, PartitionMetadataFile, RequestLocal} import kafka.utils._ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic @@ -39,11 +38,11 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET import org.apache.kafka.common.requests.ProduceResponse.RecordError -import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.common.utils.{PrimitiveRef, Time, Utils} import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_0_IV0 -import org.apache.kafka.server.log.internals.{AbortedTxn, CompletedTxn} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LogValidator} import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig import org.apache.kafka.server.record.BrokerCompressionType @@ -254,6 +253,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, /* A lock that guards all modifications to the log */ private val lock = new Object + private val validatorMetricsRecorder = newValidatorMetricsRecorder(brokerTopicStats.allTopicsStats) /* The earliest offset which is part of an incomplete transaction. This is used to compute the * last stable offset (LSO) in ReplicaManager. Note that it is possible that the "true" first unstable offset @@ -760,10 +760,10 @@ class UnifiedLog(@volatile var logStartOffset: Long, */ def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, - origin: AppendOrigin = AppendOrigin.Client, + origin: AppendOrigin = AppendOrigin.CLIENT, interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest, requestLocal: RequestLocal = RequestLocal.NoCaching): LogAppendInfo = { - val validateAndAssignOffsets = origin != AppendOrigin.RaftLeader + val validateAndAssignOffsets = origin != AppendOrigin.RAFT_LEADER append(records, origin, interBrokerProtocolVersion, validateAndAssignOffsets, leaderEpoch, Some(requestLocal), ignoreRecordSize = false) } @@ -776,7 +776,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, */ def appendAsFollower(records: MemoryRecords): LogAppendInfo = { append(records, - origin = AppendOrigin.Replication, + origin = AppendOrigin.REPLICATION, interBrokerProtocolVersion = MetadataVersion.latest, validateAndAssignOffsets = false, leaderEpoch = -1, @@ -829,15 +829,12 @@ class UnifiedLog(@volatile var logStartOffset: Long, localLog.checkIfMemoryMappedBufferClosed() if (validateAndAssignOffsets) { // assign offsets to the message set - val offset = new LongRef(localLog.logEndOffset) + val offset = PrimitiveRef.ofLong(localLog.logEndOffset) appendInfo.firstOffset = Some(LogOffsetMetadata(offset.value)) - val now = time.milliseconds val validateAndOffsetAssignResult = try { - LogValidator.validateMessagesAndAssignOffsets(validRecords, + val validator = new LogValidator(validRecords, topicPartition, - offset, time, - now, appendInfo.sourceCompression, appendInfo.targetCompression, config.compact, @@ -846,21 +843,26 @@ class UnifiedLog(@volatile var logStartOffset: Long, config.messageTimestampDifferenceMaxMs, leaderEpoch, origin, - interBrokerProtocolVersion, - brokerTopicStats, + interBrokerProtocolVersion + ) + validator.validateMessagesAndAssignOffsets(offset, + validatorMetricsRecorder, requestLocal.getOrElse(throw new IllegalArgumentException( - "requestLocal should be defined if assignOffsets is true"))) + "requestLocal should be defined if assignOffsets is true") + ).bufferSupplier + ) } catch { case e: IOException => throw new KafkaException(s"Error validating messages while appending to log $name", e) } + validRecords = validateAndOffsetAssignResult.validatedRecords - appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp - appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp + appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestampMs + appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestampMs appendInfo.lastOffset = offset.value - 1 appendInfo.recordConversionStats = validateAndOffsetAssignResult.recordConversionStats if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME) - appendInfo.logAppendTime = now + appendInfo.logAppendTime = validateAndOffsetAssignResult.logAppendTimeMs // re-validate message sizes if there's a possibility that they have changed (due to re-compression or message // format conversion) @@ -1070,7 +1072,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, if (batch.hasProducerId) { // if this is a client produce request, there will be up to 5 batches which could have been duplicated. // If we find a duplicate, we return the metadata of the appended batch to the client. - if (origin == AppendOrigin.Client) { + if (origin == AppendOrigin.CLIENT) { val maybeLastEntry = producerStateManager.lastEntry(batch.producerId) maybeLastEntry.flatMap(_.findDuplicateBatch(batch)).foreach { duplicate => @@ -1129,11 +1131,11 @@ class UnifiedLog(@volatile var logStartOffset: Long, var lastOffsetOfFirstBatch = -1L records.batches.forEach { batch => - if (origin == RaftLeader && batch.partitionLeaderEpoch != leaderEpoch) { + if (origin == AppendOrigin.RAFT_LEADER && batch.partitionLeaderEpoch != leaderEpoch) { throw new InvalidRecordException("Append from Raft leader did not set the batch epoch correctly") } // we only validate V2 and higher to avoid potential compatibility issues with older clients - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2 && origin == AppendOrigin.Client && batch.baseOffset != 0) + if (batch.magic >= RecordBatch.MAGIC_VALUE_V2 && origin == AppendOrigin.CLIENT && batch.baseOffset != 0) throw new InvalidRecordException(s"The baseOffset of the record batch in the append to $topicPartition should " + s"be 0, but it is ${batch.baseOffset}") @@ -1955,7 +1957,7 @@ object UnifiedLog extends Logging { batch, loadedProducers, firstOffsetMetadata = None, - origin = AppendOrigin.Replication) + origin = AppendOrigin.REPLICATION) maybeCompletedTxn.foreach(completedTxns += _) } } @@ -1981,8 +1983,8 @@ object UnifiedLog extends Logging { * @param dir The directory in which the log will reside * @param topicPartition The topic partition * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure - * @param recordVersion The record version - * @param logPrefix The logging prefix + * @param recordVersion The record version + * @param logPrefix The logging prefix * @return The new LeaderEpochFileCache instance (if created), none otherwise */ def maybeCreateLeaderEpochCache(dir: File, @@ -2059,7 +2061,7 @@ object UnifiedLog extends Logging { * @param time The time instance used for checking the clock * @param reloadFromCleanShutdown True if the producer state is being built after a clean shutdown, * false otherwise. - * @param logPrefix The logging prefix + * @param logPrefix The logging prefix */ private[log] def rebuildProducerState(producerStateManager: ProducerStateManager, segments: LogSegments, @@ -2165,7 +2167,9 @@ object UnifiedLog extends Logging { parentDir: String, topicPartition: TopicPartition): Unit = { val snapshotsToDelete = segments.flatMap { segment => - producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset)} + producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset) + } + def deleteProducerSnapshots(): Unit = { LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, @@ -2185,6 +2189,27 @@ object UnifiedLog extends Logging { private[log] def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = { LocalLog.createNewCleanedSegment(dir, logConfig, baseOffset) } + + // Visible for benchmarking + def newValidatorMetricsRecorder(allTopicsStats: BrokerTopicMetrics): LogValidator.MetricsRecorder = { + new LogValidator.MetricsRecorder { + def recordInvalidMagic(): Unit = + allTopicsStats.invalidMagicNumberRecordsPerSec.mark() + + def recordInvalidOffset(): Unit = + allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() + + def recordInvalidSequence(): Unit = + allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() + + def recordInvalidChecksums(): Unit = + allTopicsStats.invalidMessageCrcRecordsPerSec.mark() + + def recordNoKeyCompactedTopic(): Unit = + allTopicsStats.noKeyCompactedTopicRecordsPerSec.mark() + } + } + } object LogMetricNames { diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 2aa2feca352..6e1ff92ecd2 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -16,7 +16,7 @@ */ package kafka.raft -import kafka.log.{AppendOrigin, Defaults, LogConfig, LogOffsetSnapshot, ProducerStateManagerConfig, SnapshotGenerated, UnifiedLog} +import kafka.log.{Defaults, LogConfig, LogOffsetSnapshot, ProducerStateManagerConfig, SnapshotGenerated, UnifiedLog} import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp} import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, KafkaConfig, LogDirFailureChannel, RequestLocal} import kafka.utils.{CoreUtils, Logging, Scheduler} @@ -26,6 +26,7 @@ import org.apache.kafka.common.record.{ControlRecordUtils, MemoryRecords, Record import org.apache.kafka.common.utils.{BufferSupplier, Time} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch} +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import java.io.File @@ -79,7 +80,7 @@ final class KafkaMetadataLog private ( handleAndConvertLogAppendInfo( log.appendAsLeader(records.asInstanceOf[MemoryRecords], leaderEpoch = epoch, - origin = AppendOrigin.RaftLeader, + origin = AppendOrigin.RAFT_LEADER, requestLocal = RequestLocal.NoCaching ) ) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 776196455ad..662b61a5fd3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -23,7 +23,6 @@ import kafka.common.OffsetAndMetadata import kafka.controller.ReplicaAssignment import kafka.coordinator.group._ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} -import kafka.log.AppendOrigin import kafka.network.RequestChannel import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.metadata.ConfigRepository @@ -71,6 +70,7 @@ import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0} +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.server.record.BrokerCompressionType import java.lang.{Long => JLong} @@ -669,7 +669,7 @@ class KafkaApis(val requestChannel: RequestChannel, timeout = produceRequest.timeout.toLong, requiredAcks = produceRequest.acks, internalTopicsAllowed = internalTopicsAllowed, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = authorizedRequestInfo, requestLocal = requestLocal, responseCallback = sendResponseCallback, @@ -2333,7 +2333,7 @@ class KafkaApis(val requestChannel: RequestChannel, timeout = config.requestTimeoutMs.toLong, requiredAcks = -1, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, entriesPerPartition = controlRecords, requestLocal = requestLocal, responseCallback = maybeSendResponseCallback(producerId, marker.transactionResult)) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 7b56b6d6732..7862771e48f 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -24,7 +24,6 @@ import java.util.concurrent.locks.Lock import com.yammer.metrics.core.Meter import kafka.api._ import kafka.cluster.{BrokerEndPoint, Partition} -import kafka.common.RecordValidationException import kafka.controller.{KafkaController, StateChangeLogger} import kafka.log._ import kafka.log.remote.RemoteLogManager @@ -62,6 +61,7 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.server.common.MetadataVersion._ +import org.apache.kafka.server.log.internals.{AppendOrigin, RecordValidationException} import java.nio.file.{Files, Paths} import java.util @@ -984,7 +984,7 @@ class ReplicaManager(val config: KafkaConfig, val logStartOffset = processFailedRecord(topicPartition, rve.invalidException) val recordErrors = rve.recordErrors (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo( - logStartOffset, recordErrors, rve.invalidException.getMessage), Some(rve.invalidException))) + logStartOffset, recordErrors.asScala, rve.invalidException.getMessage), Some(rve.invalidException))) case t: Throwable => val logStartOffset = processFailedRecord(topicPartition, t) (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t))) diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala index ba2d2c43c21..1ca8334342a 100644 --- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala @@ -18,7 +18,6 @@ package kafka.server import kafka.cluster.BrokerEndPoint -import kafka.log.AppendOrigin import kafka.server.checkpoints.LazyOffsetCheckpoints import kafka.utils.{MockScheduler, MockTime, TestUtils} import org.apache.kafka.common.{Node, TopicPartition, Uuid} @@ -28,6 +27,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.api.Assertions._ import org.mockito.Mockito.mock @@ -158,7 +158,7 @@ class LocalLeaderEndPointTest { private def appendRecords(replicaManager: ReplicaManager, partition: TopicPartition, records: MemoryRecords, - origin: AppendOrigin = AppendOrigin.Client, + origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { @@ -189,4 +189,4 @@ class LocalLeaderEndPointTest { new SimpleRecord("third message".getBytes()), ) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index d627ce54a03..e5dab17e7ee 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -35,6 +35,7 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers @@ -369,7 +370,7 @@ class PartitionLockTest extends Logging { (0 until numRecords).foreach { _ => val batch = TestUtils.records(records = List(new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k2".getBytes, "v2".getBytes))) - partition.appendRecordsToLeader(batch, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) } } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 82ffee42d03..b42372cea68 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -54,6 +54,7 @@ import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -740,8 +741,8 @@ class PartitionTest extends AbstractPartitionTest { val requestLocal = RequestLocal.withThreadConfinedCaching // after makeLeader(() call, partition should know about all the replicas // append records with initial leader epoch - partition.appendRecordsToLeader(batch1, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) - partition.appendRecordsToLeader(batch2, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch1, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch2, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) assertEquals(partition.localLogOrException.logStartOffset, partition.localLogOrException.highWatermark, "Expected leader's HW not move") @@ -949,7 +950,7 @@ class PartitionTest extends AbstractPartitionTest { new SimpleRecord("k2".getBytes, "v2".getBytes), new SimpleRecord("k3".getBytes, "v3".getBytes)), baseOffset = 0L) - partition.appendRecordsToLeader(records, origin = AppendOrigin.Client, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) + partition.appendRecordsToLeader(records, origin = AppendOrigin.CLIENT, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) def fetchOffset(isolationLevel: Option[IsolationLevel], timestamp: Long): TimestampAndOffset = { val res = partition.fetchOffsetForTimestamp(timestamp, @@ -1075,9 +1076,9 @@ class PartitionTest extends AbstractPartitionTest { // after makeLeader(() call, partition should know about all the replicas // append records with initial leader epoch - val lastOffsetOfFirstBatch = partition.appendRecordsToLeader(batch1, origin = AppendOrigin.Client, + val lastOffsetOfFirstBatch = partition.appendRecordsToLeader(batch1, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal).lastOffset - partition.appendRecordsToLeader(batch2, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch2, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) assertEquals(partition.localLogOrException.logStartOffset, partition.log.get.highWatermark, "Expected leader's HW not move") // let the follower in ISR move leader's HW to move further but below LEO @@ -1109,7 +1110,7 @@ class PartitionTest extends AbstractPartitionTest { val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset // append records with the latest leader epoch - partition.appendRecordsToLeader(batch3, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch3, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) // fetch from follower not in ISR from log start offset should not add this follower to ISR fetchFollower(partition, replicaId = follower1, fetchOffset = 0) diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index d741508231f..326ca5101ce 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -21,9 +21,8 @@ import java.util.concurrent.{ConcurrentHashMap, Executors} import java.util.{Collections, Random} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.Lock - import kafka.coordinator.AbstractCoordinatorConcurrencyTest._ -import kafka.log.{AppendOrigin, LogConfig, UnifiedLog} +import kafka.log.{LogConfig, UnifiedLog} import kafka.server._ import kafka.utils._ import kafka.utils.timer.MockTimer @@ -32,6 +31,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordConversionStats} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.mockito.Mockito.{mock, withSettings, CALLS_REAL_METHODS} diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index be1f90bc735..5ee68ef3505 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -31,13 +31,13 @@ import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import kafka.cluster.Partition -import kafka.log.AppendOrigin import kafka.zk.KafkaZkClient import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -3854,7 +3854,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -3888,7 +3888,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -4032,7 +4032,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -4065,7 +4065,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index 16cdca0591d..94be08c671f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -21,12 +21,12 @@ import java.lang.management.ManagementFactory import java.nio.ByteBuffer import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, Optional} - import com.yammer.metrics.core.Gauge + import javax.management.ObjectName import kafka.cluster.Partition import kafka.common.OffsetAndMetadata -import kafka.log.{AppendOrigin, LogAppendInfo, UnifiedLog} +import kafka.log.{LogAppendInfo, UnifiedLog} import kafka.server.{FetchDataInfo, FetchLogEnd, HostedPartition, KafkaConfig, LogOffsetMetadata, ReplicaManager, RequestLocal} import kafka.utils.{KafkaScheduler, MockTime, TestUtils} import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor @@ -42,6 +42,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion._ +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -1172,7 +1173,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1208,7 +1209,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1282,7 +1283,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1322,7 +1323,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedResponseCallback.capture(), any[Option[ReentrantLock]], @@ -1380,7 +1381,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], @@ -1428,7 +1429,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], @@ -1561,7 +1562,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], @@ -1640,7 +1641,7 @@ class GroupMetadataManagerTest { time.sleep(2) when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1655,7 +1656,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1681,7 +1682,7 @@ class GroupMetadataManagerTest { when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) mockGetPartition() when(partition.appendRecordsToLeader(recordsCapture.capture(), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1724,7 +1725,7 @@ class GroupMetadataManagerTest { when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) mockGetPartition() when(partition.appendRecordsToLeader(recordsCapture.capture(), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1792,7 +1793,7 @@ class GroupMetadataManagerTest { val recordsCapture: ArgumentCaptor[MemoryRecords] = ArgumentCaptor.forClass(classOf[MemoryRecords]) when(partition.appendRecordsToLeader(recordsCapture.capture(), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1886,7 +1887,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1907,7 +1908,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1947,7 +1948,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -2020,7 +2021,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -2144,13 +2145,13 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() verify(partition).appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any()) verify(replicaManager, times(2)).onlinePartition(groupTopicPartition) @@ -2454,7 +2455,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -2469,7 +2470,7 @@ class GroupMetadataManagerTest { when(replicaManager.appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), capturedRecords.capture(), capturedCallback.capture(), any[Option[ReentrantLock]], diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 6a56b768c47..59a4ef9241b 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -20,9 +20,8 @@ import java.lang.management.ManagementFactory import java.nio.ByteBuffer import java.util.concurrent.CountDownLatch import java.util.concurrent.locks.ReentrantLock - import javax.management.ObjectName -import kafka.log.{AppendOrigin, Defaults, LogConfig, UnifiedLog} +import kafka.log.{Defaults, LogConfig, UnifiedLog} import kafka.server.{FetchDataInfo, FetchLogEnd, LogOffsetMetadata, ReplicaManager, RequestLocal} import kafka.utils.{MockScheduler, Pool, TestUtils} import kafka.zk.KafkaZkClient @@ -34,6 +33,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.MockTime +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.{ArgumentCaptor, ArgumentMatchers} @@ -648,7 +648,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -691,7 +691,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -731,7 +731,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -886,7 +886,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), recordsCapture.capture(), callbackCapture.capture(), any[Option[ReentrantLock]], @@ -1036,7 +1036,7 @@ class TransactionStateManagerTest { when(replicaManager.appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 2b4d212b605..3415eb09902 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -20,12 +20,12 @@ package kafka.log import java.io.File import java.nio.file.Files import java.util.Properties - import kafka.server.{BrokerTopicStats, LogDirFailureChannel} import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -678,7 +678,7 @@ class LogCleanerManagerTest extends Logging { log.appendAsLeader(MemoryRecords.withEndTransactionMarker(time.milliseconds(), producerId, producerEpoch, new EndTransactionMarker(ControlRecordType.ABORT, 15)), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) + origin = AppendOrigin.COORDINATOR) log.roll() log.updateHighWatermark(4L) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 071e8b8fd1c..6001d3094f8 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.AbortedTxn +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -303,11 +303,11 @@ class LogCleanerTest { val appendProducer2 = appendTransactionalAsLeader(log, producerId2, producerEpoch) def abort(producerId: Long): Unit = { - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Replication) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.REPLICATION) } def commit(producerId: Long): Unit = { - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Replication) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.REPLICATION) } // Append some transaction data (offset range in parenthesis) @@ -393,10 +393,10 @@ class LogCleanerTest { appendProducer1(Seq(1, 2)) appendProducer2(Seq(2, 3)) appendProducer1(Seq(3, 4)) - log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) - log.appendAsLeader(commitMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) + log.appendAsLeader(commitMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer1(Seq(2)) - log.appendAsLeader(commitMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) val abortedTransactions = log.collectAbortedTransactions(log.logStartOffset, log.logEndOffset) @@ -434,11 +434,11 @@ class LogCleanerTest { appendProducer2(Seq(5, 6)) appendProducer3(Seq(6, 7)) appendProducer1(Seq(7, 8)) - log.appendAsLeader(abortMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer3(Seq(8, 9)) - log.appendAsLeader(commitMarker(pid3, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(pid3, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer1(Seq(9, 10)) - log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) // we have only cleaned the records in the first segment val dirtyOffset = cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset))._1 @@ -469,9 +469,9 @@ class LogCleanerTest { appendProducer(Seq(1)) appendProducer(Seq(2, 3)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer(Seq(2)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // cannot remove the marker in this pass because there are still valid records @@ -480,7 +480,7 @@ class LogCleanerTest { assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) appendProducer(Seq(1, 3)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction @@ -517,10 +517,10 @@ class LogCleanerTest { val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch) appendProducer(Seq(1)) - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer(Seq(2)) appendProducer(Seq(2)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) @@ -552,7 +552,7 @@ class LogCleanerTest { // [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}] producer2(Seq(2, 3)) // offsets 2, 3 log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 4 + origin = AppendOrigin.COORDINATOR) // offset 4 log.roll() // [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}], [{2}, {3}, {Producer1: Commit}] @@ -560,7 +560,7 @@ class LogCleanerTest { log.appendAsLeader(record(2, 2), leaderEpoch = 0) // offset 5 log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 6 log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 7 + origin = AppendOrigin.COORDINATOR) // offset 7 log.roll() // first time through the records are removed @@ -582,7 +582,7 @@ class LogCleanerTest { // {1}, {3}, {4}, {5}, {6}, {7}, {8}, {9} ==> Offsets producer2(Seq(1)) // offset 8 log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 9 + origin = AppendOrigin.COORDINATOR) // offset 9 log.roll() // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] @@ -612,7 +612,7 @@ class LogCleanerTest { // [{Producer1: Commit}, {2}, {3}] log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 1 + origin = AppendOrigin.COORDINATOR) // offset 1 log.appendAsLeader(record(2, 2), leaderEpoch = 0) // offset 2 log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 3 log.roll() @@ -648,7 +648,7 @@ class LogCleanerTest { appendTransaction(Seq(1)) log.roll() - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // Both the record and the marker should remain after cleaning @@ -671,7 +671,7 @@ class LogCleanerTest { appendTransaction(Seq(1)) log.roll() - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // Both the batch and the marker should remain after cleaning. The batch is retained @@ -701,9 +701,9 @@ class LogCleanerTest { appendProducer(Seq(1)) appendProducer(Seq(2, 3)) - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer(Seq(3)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // Aborted records are removed, but the abort marker is still preserved. @@ -731,14 +731,14 @@ class LogCleanerTest { val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, - origin = AppendOrigin.Replication) + origin = AppendOrigin.REPLICATION) appendFirstTransaction(Seq(1)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) val appendSecondTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, - origin = AppendOrigin.Replication) + origin = AppendOrigin.REPLICATION) appendSecondTransaction(Seq(2)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.appendAsLeader(record(1, 1), leaderEpoch = 0) log.appendAsLeader(record(2, 1), leaderEpoch = 0) @@ -771,7 +771,7 @@ class LogCleanerTest { val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch) appendProducer(Seq(2, 3)) // batch last offset is 1 - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() def assertAbortedTransactionIndexed(): Unit = { @@ -1012,7 +1012,7 @@ class LogCleanerTest { appendProducer(Seq(1)) appendProducer(Seq(2, 3)) - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) @@ -1046,7 +1046,7 @@ class LogCleanerTest { val producerId1 = 1L val appendProducer = appendTransactionalAsLeader(log, producerId1, producerEpoch, leaderEpoch) appendProducer(Seq(1)) - log.appendAsLeader(commitMarker(producerId1, producerEpoch), leaderEpoch, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId1, producerEpoch), leaderEpoch, origin = AppendOrigin.COORDINATOR) // Now we append one transaction with a key which conflicts with the COMMIT marker appended above def commitRecordKey(): ByteBuffer = { @@ -1065,8 +1065,8 @@ class LogCleanerTest { 0, new SimpleRecord(time.milliseconds(), commitRecordKey(), ByteBuffer.wrap("foo".getBytes)) ) - log.appendAsLeader(records, leaderEpoch, origin = AppendOrigin.Client) - log.appendAsLeader(commitMarker(producerId2, producerEpoch), leaderEpoch, origin = AppendOrigin.Coordinator) + log.appendAsLeader(records, leaderEpoch, origin = AppendOrigin.CLIENT) + log.appendAsLeader(commitMarker(producerId2, producerEpoch), leaderEpoch, origin = AppendOrigin.COORDINATOR) log.roll() assertEquals(List(0, 1, 2, 3), offsetsInLog(log)) @@ -2013,7 +2013,7 @@ class LogCleanerTest { producerId: Long, producerEpoch: Short, leaderEpoch: Int = 0, - origin: AppendOrigin = AppendOrigin.Client + origin: AppendOrigin = AppendOrigin.CLIENT ): Seq[Int] => LogAppendInfo = { appendIdempotentAsLeader( log, @@ -2031,7 +2031,7 @@ class LogCleanerTest { producerEpoch: Short, isTransactional: Boolean = false, leaderEpoch: Int = 0, - origin: AppendOrigin = AppendOrigin.Client + origin: AppendOrigin = AppendOrigin.CLIENT ): Seq[Int] => LogAppendInfo = { var sequence = 0 keys: Seq[Int] => { diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index b4ff5a090c4..489bf97d9a0 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -32,7 +32,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import java.nio.file.Files import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import kafka.log -import org.apache.kafka.server.log.internals.{AbortedTxn, LazyIndex, TransactionIndex} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LazyIndex, TransactionIndex} import scala.collection.Iterable import scala.jdk.CollectionConverters._ @@ -215,7 +215,7 @@ object LogTestUtils { leaderEpoch: Int = 0): LogAppendInfo = { val records = endTxnRecords(controlType, producerId, producerEpoch, coordinatorEpoch = coordinatorEpoch, timestamp = timestamp) - log.appendAsLeader(records, origin = AppendOrigin.Coordinator, leaderEpoch = leaderEpoch) + log.appendAsLeader(records, origin = AppendOrigin.COORDINATOR, leaderEpoch = leaderEpoch) } private def endTxnRecords(controlRecordType: ControlRecordType, diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala index ac8718d3662..aa5843b7c03 100644 --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala @@ -18,16 +18,16 @@ package kafka.log import java.nio.ByteBuffer import java.util.concurrent.TimeUnit - -import kafka.common.{LongRef, RecordValidationException} -import kafka.log.LogValidator.ValidationAndOffsetAssignResult import kafka.server.{BrokerTopicStats, RequestLocal} +import kafka.utils.MockTime import kafka.utils.TestUtils.meterCount import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException} import org.apache.kafka.common.record._ -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.utils.{PrimitiveRef, Time} import org.apache.kafka.common.{InvalidRecordException, TopicPartition} import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.LogValidator.ValidationResult +import org.apache.kafka.server.log.internals.{AppendOrigin, LogValidator, RecordValidationException} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ @@ -39,8 +39,8 @@ class LogValidatorTest { val time = Time.SYSTEM val topicPartition = new TopicPartition("topic", 0) - val brokerTopicStats = new BrokerTopicStats val metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala + val metricsRecorder = UnifiedLog.newValidatorMetricsRecorder(new BrokerTopicStats().allTopicsStats) @Test def testOnlyOneBatch(): Unit = { @@ -112,23 +112,22 @@ class LogValidatorTest { private def validateMessages(records: MemoryRecords, magic: Byte, sourceCompressionType: CompressionType, - targetCompressionType: CompressionType): ValidationAndOffsetAssignResult = { - LogValidator.validateMessagesAndAssignOffsets(records, + targetCompressionType: CompressionType): ValidationResult = { + val mockTime = new MockTime(0L, 0L) + new LogValidator(records, topicPartition, - new LongRef(0L), - time, - now = 0L, + mockTime, sourceCompressionType, targetCompressionType, - compactedTopic = false, + false, magic, TimestampType.CREATE_TIME, 1000L, RecordBatch.NO_PRODUCER_EPOCH, - origin = AppendOrigin.Client, - MetadataVersion.IBP_2_3_IV1, - brokerTopicStats, - RequestLocal.withThreadConfinedCaching) + AppendOrigin.CLIENT, + MetadataVersion.IBP_2_3_IV1 + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier) } @Test @@ -142,34 +141,39 @@ class LogValidatorTest { } private def checkLogAppendTimeNonCompressed(magic: Byte): Unit = { - val now = System.currentTimeMillis() + val mockTime = new MockTime // The timestamps should be overwritten val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.NONE) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val offsetCounter = PrimitiveRef.ofLong(0) + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time= time, - now = now, - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + mockTime, + CompressionType.NONE, + CompressionType.NONE, + false, + magic, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + offsetCounter, + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) + + assertEquals(offsetCounter.value, records.records.asScala.size) val validatedRecords = validatedResults.validatedRecords assertEquals(records.records.asScala.size, validatedRecords.records.asScala.size, "message set size should not change") + val now = mockTime.milliseconds validatedRecords.batches.forEach(batch => validateLogAppendTime(now, 1234L, batch)) - assertEquals(now, validatedResults.maxTimestamp, s"Max timestamp should be $now") + assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") // 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.shallowOffsetOfMaxTimestamp, + assertEquals(expectedMaxTimestampOffset, validatedResults.shallowOffsetOfMaxTimestampMs, s"The offset of max timestamp should be $expectedMaxTimestampOffset") verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 0, records, compressed = false) @@ -181,36 +185,38 @@ class LogValidatorTest { } private def checkLogAppendTimeWithRecompression(targetMagic: Byte): Unit = { - val now = System.currentTimeMillis() + val mockTime = new MockTime // The timestamps should be overwritten val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets( + val validatedResults = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = now, - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = targetMagic, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) - val validatedRecords = validatedResults.validatedRecords + mockTime, + CompressionType.GZIP, + CompressionType.GZIP, + false, + targetMagic, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) + val validatedRecords = validatedResults.validatedRecords assertEquals(records.records.asScala.size, validatedRecords.records.asScala.size, "message set size should not change") + val now = mockTime.milliseconds() validatedRecords.batches.forEach(batch => validateLogAppendTime(now, -1, batch)) assertTrue(validatedRecords.batches.iterator.next().isValid, "MessageSet should still valid") - assertEquals(now, validatedResults.maxTimestamp, + assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") - assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"The offset of max timestamp should be ${records.records.asScala.size - 1}") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size may have been changed") @@ -230,36 +236,38 @@ class LogValidatorTest { } private def checkLogAppendTimeWithoutRecompression(magic: Byte): Unit = { - val now = System.currentTimeMillis() + val mockTime = new MockTime // The timestamps should be overwritten val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.GZIP) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets( + val validatedResults = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = now, - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + mockTime, + CompressionType.GZIP, + CompressionType.GZIP, + false, + magic, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords assertEquals(records.records.asScala.size, validatedRecords.records.asScala.size, "message set size should not change") + val now = mockTime.milliseconds() validatedRecords.batches.forEach(batch => validateLogAppendTime(now, 1234L, batch)) assertTrue(validatedRecords.batches.iterator.next().isValid, "MessageSet should still valid") - assertEquals(now, validatedResults.maxTimestamp, + assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") - assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"The offset of max timestamp should be ${records.records.asScala.size - 1}") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") @@ -296,23 +304,24 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE) records.buffer.putInt(DefaultRecordBatch.RECORDS_COUNT_OFFSET, count) records.buffer.putInt(DefaultRecordBatch.LAST_OFFSET_DELTA_OFFSET, lastOffsetDelta) - LogValidator.validateMessagesAndAssignOffsets( + new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = time.milliseconds(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) } @Test @@ -342,22 +351,25 @@ class LogValidatorTest { new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes)) - val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records, + val offsetCounter = PrimitiveRef.ofLong(0); + val validatingResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = partitionLeaderEpoch, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + magic, + TimestampType.CREATE_TIME, + 1000L, + partitionLeaderEpoch, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + offsetCounter, + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) + val validatedRecords = validatingResults.validatedRecords var i = 0 @@ -377,9 +389,11 @@ class LogValidatorTest { i += 1 } } - assertEquals(now + 1, validatingResults.maxTimestamp, + + assertEquals(i, offsetCounter.value); + assertEquals(now + 1, validatingResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") - assertEquals(1, validatingResults.shallowOffsetOfMaxTimestamp, + assertEquals(1, validatingResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be 1") assertFalse(validatingResults.messageSizeMaybeChanged, "Message size should not have been changed") @@ -415,22 +429,23 @@ class LogValidatorTest { new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes)) - val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatingResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = partitionLeaderEpoch, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.GZIP, + false, + magic, + TimestampType.CREATE_TIME, + 1000L, + partitionLeaderEpoch, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatingResults.validatedRecords var i = 0 @@ -449,9 +464,9 @@ class LogValidatorTest { i += 1 } } - assertEquals(now + 1, validatingResults.maxTimestamp, + assertEquals(now + 1, validatingResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") - assertEquals(2, validatingResults.shallowOffsetOfMaxTimestamp, + assertEquals(2, validatingResults.shallowOffsetOfMaxTimestampMs, "Offset of max timestamp should be 2") assertTrue(validatingResults.messageSizeMaybeChanged, "Message size should have been changed") @@ -472,22 +487,23 @@ class LogValidatorTest { private def checkCreateTimeUpConversionFromV0(toMagic: Byte): Unit = { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = toMagic, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + toMagic, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords for (batch <- validatedRecords.batches.asScala) { @@ -499,9 +515,9 @@ class LogValidatorTest { assertEquals(RecordBatch.NO_PRODUCER_ID, batch.producerId) assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence) } - assertEquals(validatedResults.maxTimestamp, RecordBatch.NO_TIMESTAMP, + assertEquals(validatedResults.maxTimestampMs, RecordBatch.NO_TIMESTAMP, s"Max timestamp should be ${RecordBatch.NO_TIMESTAMP}") - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed") @@ -518,22 +534,23 @@ class LogValidatorTest { def testCreateTimeUpConversionV1ToV2(): Unit = { val timestamp = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP, timestamp = timestamp) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = timestamp, - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V2, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest, + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords for (batch <- validatedRecords.batches.asScala) { @@ -545,8 +562,8 @@ class LogValidatorTest { assertEquals(RecordBatch.NO_PRODUCER_ID, batch.producerId) assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence) } - assertEquals(timestamp, validatedResults.maxTimestamp) - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(timestamp, validatedResults.maxTimestampMs) + assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed") @@ -576,22 +593,23 @@ class LogValidatorTest { new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes)) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = magic, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = partitionLeaderEpoch, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + magic, + TimestampType.CREATE_TIME, + 1000L, + partitionLeaderEpoch, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords var i = 0 @@ -610,8 +628,8 @@ class LogValidatorTest { i += 1 } } - assertEquals(now + 1, validatedResults.maxTimestamp, s"Max timestamp should be ${now + 1}") - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(now + 1, validatedResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") + assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") @@ -629,23 +647,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L, codec = CompressionType.NONE) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -653,23 +672,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, codec = CompressionType.NONE) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -677,23 +697,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L, codec = CompressionType.GZIP) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V1, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -701,23 +722,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, codec = CompressionType.GZIP) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V1, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -725,22 +747,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) val offset = 1234567 checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - magic = RecordBatch.MAGIC_VALUE_V0, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest, + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -748,22 +769,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -772,22 +792,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.NONE) val offset = 1234567 checkOffsets(records, 0) - val messageWithOffset = LogValidator.validateMessagesAndAssignOffsets(records, + val messageWithOffset = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(messageWithOffset, offset) } @@ -797,22 +816,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.NONE) val offset = 1234567 checkOffsets(records, 0) - val messageWithOffset = LogValidator.validateMessagesAndAssignOffsets(records, + val messageWithOffset = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(messageWithOffset, offset) } @@ -822,23 +840,22 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val compressedMessagesWithOffset = LogValidator.validateMessagesAndAssignOffsets( + val compressedMessagesWithOffset = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(compressedMessagesWithOffset, offset) } @@ -848,23 +865,22 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val compressedMessagesWithOffset = LogValidator.validateMessagesAndAssignOffsets( + val compressedMessagesWithOffset = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(compressedMessagesWithOffset, offset) } @@ -873,22 +889,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) checkOffsets(records, 0) val offset = 1234567 - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = false) @@ -899,22 +914,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) checkOffsets(records, 0) val offset = 1234567 - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = false) @@ -925,22 +939,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = true) @@ -951,22 +964,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = true) @@ -977,22 +989,21 @@ class LogValidatorTest { val offset = 1234567 val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0) val records = MemoryRecords.withEndTransactionMarker(23423L, 5, endTxnMarker) - assertThrows(classOf[InvalidRecordException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[InvalidRecordException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.CURRENT_MAGIC_VALUE, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.CURRENT_MAGIC_VALUE, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1000,22 +1011,21 @@ class LogValidatorTest { val offset = 1234567 val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0) val records = MemoryRecords.withEndTransactionMarker(23423L, 5, endTxnMarker) - val result = LogValidator.validateMessagesAndAssignOffsets(records, + val result = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.SNAPPY, - compactedTopic = false, - magic = RecordBatch.CURRENT_MAGIC_VALUE, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Coordinator, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.SNAPPY, + false, + RecordBatch.CURRENT_MAGIC_VALUE, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.COORDINATOR, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val batches = TestUtils.toList(result.validatedRecords.batches) assertEquals(1, batches.size) val batch = batches.get(0) @@ -1028,22 +1038,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, codec = CompressionType.NONE) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1052,22 +1061,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, CompressionType.GZIP) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1075,22 +1083,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.NONE) checkOffsets(records, 0) val offset = 1234567 - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1098,22 +1105,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1122,22 +1128,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1146,22 +1151,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1172,22 +1176,21 @@ class LogValidatorTest { val sequence = 0 val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes)) - assertThrows(classOf[UnsupportedForMessageFormatException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1198,22 +1201,21 @@ class LogValidatorTest { val sequence = 0 val records = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes)) - assertThrows(classOf[UnsupportedForMessageFormatException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1222,22 +1224,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1246,45 +1247,42 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test def testNonIncreasingOffsetRecordBatchHasMetricsLogged(): Unit = { val records = createNonIncreasingOffsetRecords(RecordBatch.MAGIC_VALUE_V2) records.batches().asScala.head.setLastOffset(2) - assertThrows(classOf[InvalidRecordException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[InvalidRecordException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0L), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) - ) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec}")), 1) assertTrue(meterCount(s"${BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec}") > 0) } @@ -1296,25 +1294,23 @@ class LogValidatorTest { @Test def testZStdCompressedWithUnavailableIBPVersion(): Unit = { - val now = System.currentTimeMillis() // The timestamps should be overwritten val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE) - assertThrows(classOf[UnsupportedCompressionTypeException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[UnsupportedCompressionTypeException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time= time, - now = now, - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.ZSTD, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.IBP_2_0_IV1, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.ZSTD, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.IBP_2_0_IV1 + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1333,27 +1329,26 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, codec = CompressionType.GZIP) val e = assertThrows(classOf[RecordValidationException], - () => LogValidator.validateMessagesAndAssignOffsets( + () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V1, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) ) assertTrue(e.invalidException.isInstanceOf[InvalidTimestampException]) - assertTrue(e.recordErrors.nonEmpty) + assertFalse(e.recordErrors.isEmpty) assertEquals(e.recordErrors.size, 3) } @@ -1366,10 +1361,10 @@ class LogValidatorTest { ) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) - assertTrue(e.recordErrors.nonEmpty) + assertFalse(e.recordErrors.isEmpty) // recordsWithInvalidInnerMagic creates 20 records assertEquals(e.recordErrors.size, 20) - e.recordErrors.foreach(assertNotNull(_)) + e.recordErrors.asScala.foreach(assertNotNull(_)) } @Test @@ -1399,7 +1394,7 @@ class LogValidatorTest { // if there is a mix of both regular InvalidRecordException and InvalidTimestampException, // InvalidTimestampException takes precedence assertTrue(e.invalidException.isInstanceOf[InvalidTimestampException]) - assertTrue(e.recordErrors.nonEmpty) + assertFalse(e.recordErrors.isEmpty) assertEquals(6, e.recordErrors.size) } @@ -1413,22 +1408,21 @@ class LogValidatorTest { isTransactional, false) buffer.flip() val records = MemoryRecords.readableRecords(buffer) - assertThrows(classOf[InvalidRecordException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[InvalidRecordException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = sourceCompression, - targetCompression = targetCompression, - compactedTopic = false, - magic = RecordBatch.CURRENT_MAGIC_VALUE, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + sourceCompression, + targetCompression, + false, + RecordBatch.CURRENT_MAGIC_VALUE, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } private def createRecords(magicValue: Byte, diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala index c808d03a72e..359c46de0b7 100644 --- a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{MockTime, Utils} -import org.apache.kafka.server.log.internals.CompletedTxn +import org.apache.kafka.server.log.internals.{AppendOrigin, CompletedTxn} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.Mockito.{mock, when} @@ -110,7 +110,7 @@ class ProducerStateManagerTest { val epoch = 15.toShort val sequence = Int.MaxValue val offset = 735L - append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.REPLICATION) append(stateManager, producerId, epoch, 0, offset + 500) @@ -128,7 +128,7 @@ class ProducerStateManagerTest { def testProducerSequenceWithWrapAroundBatchRecord(): Unit = { val epoch = 15.toShort - val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Replication) + val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.REPLICATION) // Sequence number wrap around appendInfo.appendDataBatch(epoch, Int.MaxValue - 10, 9, time.milliseconds(), LogOffsetMetadata(2000L), 2020L, isTransactional = false) @@ -148,7 +148,7 @@ class ProducerStateManagerTest { val epoch = 15.toShort val sequence = Int.MaxValue val offset = 735L - append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.REPLICATION) assertThrows(classOf[OutOfOrderSequenceException], () => append(stateManager, producerId, epoch, 1, offset + 500)) } @@ -157,7 +157,7 @@ class ProducerStateManagerTest { val epoch = 5.toShort val sequence = 16 val offset = 735L - append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.REPLICATION) val maybeLastEntry = stateManager.lastEntry(producerId) assertTrue(maybeLastEntry.isDefined) @@ -197,7 +197,7 @@ class ProducerStateManagerTest { val producerEpoch = 0.toShort val offset = 992342L val seq = 0 - val producerAppendInfo = new ProducerAppendInfo(partition, producerId, ProducerStateEntry.empty(producerId), AppendOrigin.Client) + val producerAppendInfo = new ProducerAppendInfo(partition, producerId, ProducerStateEntry.empty(producerId), AppendOrigin.CLIENT) val firstOffsetMetadata = LogOffsetMetadata(messageOffset = offset, segmentBaseOffset = 990000L, relativePositionInSegment = 234224) @@ -235,7 +235,7 @@ class ProducerStateManagerTest { } // Start one transaction in a separate append - val firstAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val firstAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) appendData(16L, 20L, firstAppend) assertEquals(new TxnMetadata(producerId, 16L), firstAppend.startedTransactions.head) stateManager.update(firstAppend) @@ -245,7 +245,7 @@ class ProducerStateManagerTest { // Now do a single append which completes the old transaction, mixes in // some empty transactions, one non-empty complete transaction, and one // incomplete transaction - val secondAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val secondAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) val firstCompletedTxn = appendEndTxn(ControlRecordType.COMMIT, 21, secondAppend) assertEquals(Some(new CompletedTxn(producerId, 16L, 21, false)), firstCompletedTxn) assertEquals(None, appendEndTxn(ControlRecordType.COMMIT, 22, secondAppend)) @@ -371,7 +371,7 @@ class ProducerStateManagerTest { partition, producerId, ProducerStateEntry.empty(producerId), - AppendOrigin.Client + AppendOrigin.CLIENT ) val firstOffsetMetadata = LogOffsetMetadata(messageOffset = startOffset, segmentBaseOffset = segmentBaseOffset, relativePositionInSegment = 50 * relativeOffset) @@ -418,14 +418,14 @@ class ProducerStateManagerTest { def testPrepareUpdateDoesNotMutate(): Unit = { val producerEpoch = 0.toShort - val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) appendInfo.appendDataBatch(producerEpoch, 0, 5, time.milliseconds(), LogOffsetMetadata(15L), 20L, isTransactional = false) assertEquals(None, stateManager.lastEntry(producerId)) stateManager.update(appendInfo) assertTrue(stateManager.lastEntry(producerId).isDefined) - val nextAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val nextAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) nextAppendInfo.appendDataBatch(producerEpoch, 6, 10, time.milliseconds(), LogOffsetMetadata(26L), 30L, isTransactional = false) assertTrue(stateManager.lastEntry(producerId).isDefined) @@ -449,7 +449,7 @@ class ProducerStateManagerTest { val offset = 9L append(stateManager, producerId, producerEpoch, 0, offset) - val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) appendInfo.appendDataBatch(producerEpoch, 1, 5, time.milliseconds(), LogOffsetMetadata(16L), 20L, isTransactional = true) var lastEntry = appendInfo.toEntry @@ -670,7 +670,7 @@ class ProducerStateManagerTest { // entry added after recovery. The pid should be expired now, and would not exist in the pid mapping. Nonetheless // the append on a replica should be accepted with the local producer state updated to the appended value. assertFalse(recoveredMapping.activeProducers.contains(producerId)) - append(recoveredMapping, producerId, epoch, sequence, 2L, 70001, origin = AppendOrigin.Replication) + append(recoveredMapping, producerId, epoch, sequence, 2L, 70001, origin = AppendOrigin.REPLICATION) assertTrue(recoveredMapping.activeProducers.contains(producerId)) val producerStateEntry = recoveredMapping.activeProducers.get(producerId).head assertEquals(epoch, producerStateEntry.producerEpoch) @@ -685,10 +685,10 @@ class ProducerStateManagerTest { val outOfOrderSequence = 3 // First we ensure that we raise an OutOfOrderSequenceException is raised when the append comes from a client. - assertThrows(classOf[OutOfOrderSequenceException], () => append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.Client)) + assertThrows(classOf[OutOfOrderSequenceException], () => append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.CLIENT)) assertEquals(0L, stateManager.activeProducers(producerId).lastSeq) - append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.REPLICATION) assertEquals(outOfOrderSequence, stateManager.activeProducers(producerId).lastSeq) } @@ -905,9 +905,9 @@ class ProducerStateManagerTest { val epoch = 0.toShort append(stateManager, producerId, epoch, RecordBatch.NO_SEQUENCE, offset = 99, - isTransactional = true, origin = AppendOrigin.Coordinator) + isTransactional = true, origin = AppendOrigin.COORDINATOR) append(stateManager, producerId, epoch, RecordBatch.NO_SEQUENCE, offset = 100, - isTransactional = true, origin = AppendOrigin.Coordinator) + isTransactional = true, origin = AppendOrigin.COORDINATOR) } @Test @@ -994,7 +994,7 @@ class ProducerStateManagerTest { when(batch.iterator).thenReturn(Collections.emptyIterator[Record]) // Appending the empty control batch should not throw and a new transaction shouldn't be started - append(stateManager, producerId, baseOffset, batch, origin = AppendOrigin.Client) + append(stateManager, producerId, baseOffset, batch, origin = AppendOrigin.CLIENT) assertEquals(None, stateManager.lastEntry(producerId).get.currentTxnFirstOffset) } @@ -1102,7 +1102,7 @@ class ProducerStateManagerTest { offset: Long, coordinatorEpoch: Int = 0, timestamp: Long = time.milliseconds()): Option[CompletedTxn] = { - val producerAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Coordinator) + val producerAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.COORDINATOR) val endTxnMarker = new EndTransactionMarker(controlType, coordinatorEpoch) val completedTxnOpt = producerAppendInfo.appendEndTxnMarker(endTxnMarker, producerEpoch, offset, timestamp) mapping.update(producerAppendInfo) @@ -1118,7 +1118,7 @@ class ProducerStateManagerTest { offset: Long, timestamp: Long = time.milliseconds(), isTransactional: Boolean = false, - origin : AppendOrigin = AppendOrigin.Client): Unit = { + origin : AppendOrigin = AppendOrigin.CLIENT): Unit = { val producerAppendInfo = stateManager.prepareUpdate(producerId, origin) producerAppendInfo.appendDataBatch(producerEpoch, seq, seq, timestamp, LogOffsetMetadata(offset), offset, isTransactional) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 45335eec8ec..4a280795017 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.nio.file.Files import java.util.concurrent.{Callable, ConcurrentHashMap, Executors} import java.util.{Optional, Properties} -import kafka.common.{OffsetsOutOfOrderException, RecordValidationException, UnexpectedAppendOffsetException} +import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.remote.RemoteLogManager import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} @@ -37,7 +37,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} -import org.apache.kafka.server.log.internals.AbortedTxn +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, RecordValidationException} import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ @@ -127,16 +127,16 @@ class UnifiedLogTest { new SimpleRecord(mockTime.milliseconds, "c".getBytes, "value".getBytes) ), baseOffset = offset, partitionLeaderEpoch = leaderEpoch) - log.appendAsLeader(records(0), leaderEpoch, AppendOrigin.RaftLeader) + log.appendAsLeader(records(0), leaderEpoch, AppendOrigin.RAFT_LEADER) assertEquals(0, log.logStartOffset) assertEquals(3L, log.logEndOffset) // Since raft leader is responsible for assigning offsets, and the LogValidator is bypassed from the performance perspective, // so the first offset of the MemoryRecords to be append should equal to the next offset in the log - assertThrows(classOf[UnexpectedAppendOffsetException], () => (log.appendAsLeader(records(1), leaderEpoch, AppendOrigin.RaftLeader))) + assertThrows(classOf[UnexpectedAppendOffsetException], () => log.appendAsLeader(records(1), leaderEpoch, AppendOrigin.RAFT_LEADER)) // When the first offset of the MemoryRecords to be append equals to the next offset in the log, append will succeed - log.appendAsLeader(records(3), leaderEpoch, AppendOrigin.RaftLeader) + log.appendAsLeader(records(3), leaderEpoch, AppendOrigin.RAFT_LEADER) assertEquals(6, log.logEndOffset) } @@ -1788,22 +1788,22 @@ class UnifiedLogTest { () => log.appendAsLeader(messageSetWithUnkeyedMessage, leaderEpoch = 0)) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertEquals(1, e.recordErrors.size) - assertEquals(0, e.recordErrors.head.batchIndex) - assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix)) + assertEquals(0, e.recordErrors.get(0).batchIndex) + assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix)) e = assertThrows(classOf[RecordValidationException], () => log.appendAsLeader(messageSetWithOneUnkeyedMessage, leaderEpoch = 0)) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertEquals(1, e.recordErrors.size) - assertEquals(0, e.recordErrors.head.batchIndex) - assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix)) + assertEquals(0, e.recordErrors.get(0).batchIndex) + assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix)) e = assertThrows(classOf[RecordValidationException], () => log.appendAsLeader(messageSetWithCompressedUnkeyedMessage, leaderEpoch = 0)) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertEquals(1, e.recordErrors.size) - assertEquals(1, e.recordErrors.head.batchIndex) // batch index is 1 - assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix)) + assertEquals(1, e.recordErrors.get(0).batchIndex) // batch index is 1 + assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix)) // check if metric for NoKeyCompactedTopicRecordsPerSec is logged assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec}")), 1) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 848f3542549..4b039b5543f 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -28,7 +28,6 @@ import kafka.cluster.Broker import kafka.controller.{ControllerContext, KafkaController} import kafka.coordinator.group._ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} -import kafka.log.AppendOrigin import kafka.network.RequestChannel import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.{ConfigRepository, KRaftMetadataCache, MockConfigRepository, ZkMetadataCache} @@ -93,6 +92,7 @@ import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartiti import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} +import org.apache.kafka.server.log.internals.AppendOrigin class KafkaApisTest { private val requestChannel: RequestChannel = mock(classOf[RequestChannel]) @@ -1645,7 +1645,7 @@ class KafkaApisTest { when(replicaManager.appendRecords(anyLong, anyShort, ArgumentMatchers.eq(false), - ArgumentMatchers.eq(AppendOrigin.Client), + ArgumentMatchers.eq(AppendOrigin.CLIENT), any(), responseCallback.capture(), any(), @@ -1786,7 +1786,7 @@ class KafkaApisTest { when(replicaManager.appendRecords(anyLong, anyShort, ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), responseCallback.capture(), any(), @@ -1916,7 +1916,7 @@ class KafkaApisTest { when(replicaManager.appendRecords(anyLong, anyShort, ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), responseCallback.capture(), any(), @@ -1948,7 +1948,7 @@ class KafkaApisTest { verify(replicaManager).appendRecords(anyLong, anyShort, ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any(), diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index 559485ff4dd..df41bafb315 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -21,7 +21,7 @@ import java.util import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit} import java.util.{Optional, Properties} import kafka.api.LeaderAndIsr -import kafka.log.{AppendOrigin, LogConfig} +import kafka.log.LogConfig import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.MockConfigRepository import kafka.utils.TestUtils.waitUntilTrue @@ -39,6 +39,7 @@ import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.PartitionRegistration +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.Mockito @@ -283,7 +284,7 @@ class ReplicaManagerConcurrencyTest { timeout = 30000, requiredAcks = (-1).toShort, internalTopicsAllowed = false, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = collection.Map(topicPartition -> TestUtils.records(records)), responseCallback = produceCallback ) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index e623816c397..1958233184c 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -58,6 +58,7 @@ import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, C import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -178,7 +179,7 @@ class ReplicaManagerTest { timeout = 0, requiredAcks = 3, internalTopicsAllowed = false, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("first message".getBytes))), responseCallback = callback) @@ -456,7 +457,7 @@ class ReplicaManagerTest { val abortTxnMarker = new EndTransactionMarker(ControlRecordType.ABORT, 0) val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, abortTxnMarker) appendRecords(replicaManager, new TopicPartition(topic, 0), - abortRecordBatch, origin = AppendOrigin.Coordinator).onFire { response => + abortRecordBatch, origin = AppendOrigin.COORDINATOR).onFire { response => assertEquals(Errors.NONE, response.error) } assertLateTransactionCount(Some(0)) @@ -550,7 +551,7 @@ class ReplicaManagerTest { val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0) val commitRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker) appendRecords(replicaManager, new TopicPartition(topic, 0), commitRecordBatch, - origin = AppendOrigin.Coordinator) + origin = AppendOrigin.COORDINATOR) .onFire { response => assertEquals(Errors.NONE, response.error) } // the LSO has advanced, but the appended commit marker has not been replicated, so @@ -635,7 +636,7 @@ class ReplicaManagerTest { val endTxnMarker = new EndTransactionMarker(ControlRecordType.ABORT, 0) val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker) appendRecords(replicaManager, new TopicPartition(topic, 0), abortRecordBatch, - origin = AppendOrigin.Coordinator) + origin = AppendOrigin.COORDINATOR) .onFire { response => assertEquals(Errors.NONE, response.error) } // fetch as follower to advance the high watermark @@ -1510,7 +1511,7 @@ class ReplicaManagerTest { val simpleRecords = Seq(new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)) val appendResult = appendRecords(replicaManager, tp0, - MemoryRecords.withRecords(CompressionType.NONE, simpleRecords.toSeq: _*), AppendOrigin.Client) + MemoryRecords.withRecords(CompressionType.NONE, simpleRecords.toSeq: _*), AppendOrigin.CLIENT) // Increment the hw in the leader by fetching from the last offset val fetchOffset = simpleRecords.size @@ -1929,7 +1930,7 @@ class ReplicaManagerTest { timeout = 10, requiredAcks = -1, internalTopicsAllowed = false, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = Map(topicPartition -> records), responseCallback = callback ) @@ -2149,7 +2150,7 @@ class ReplicaManagerTest { private def appendRecords(replicaManager: ReplicaManager, partition: TopicPartition, records: MemoryRecords, - origin: AppendOrigin = AppendOrigin.Client, + origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { @@ -2786,7 +2787,7 @@ class ReplicaManagerTest { val batch = TestUtils.records(records = List( new SimpleRecord(10, "k1".getBytes, "v1".getBytes), new SimpleRecord(11, "k2".getBytes, "v2".getBytes))) - partition.appendRecordsToLeader(batch, AppendOrigin.Client, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) + partition.appendRecordsToLeader(batch, AppendOrigin.CLIENT, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) partition.log.get.updateHighWatermark(2L) partition.log.get.maybeIncrementLogStartOffset(1L, LeaderOffsetIncremented) replicaManager.logManager.checkpointLogRecoveryOffsets() diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 08acbf3827a..864c5436b33 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -21,8 +21,7 @@ import java.io.{ByteArrayOutputStream, File, PrintWriter} import java.nio.ByteBuffer import java.util import java.util.Properties - -import kafka.log.{AppendOrigin, Defaults, LogConfig, LogTestUtils, ProducerStateManagerConfig, UnifiedLog} +import kafka.log.{Defaults, LogConfig, LogTestUtils, ProducerStateManagerConfig, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} import kafka.server.{BrokerTopicStats, FetchLogEnd, KafkaRaftServer, LogDirFailureChannel} import kafka.tools.DumpLogSegments.TimeIndexDumpErrors @@ -36,6 +35,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch} import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.snapshot.RecordsSnapshotWriter import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -136,7 +136,7 @@ class DumpLogSegmentsTest { log.appendAsLeader(MemoryRecords.withEndTransactionMarker(98323L, 99.toShort, new EndTransactionMarker(ControlRecordType.COMMIT, 100) - ), origin = AppendOrigin.Coordinator, leaderEpoch = 7) + ), origin = AppendOrigin.COORDINATOR, leaderEpoch = 7) assertDumpLogRecordMetadata() } diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index 93702b7b9db..4c3a5fad94c 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -495,6 +495,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + + + + + + + diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java index e9910da57c0..0b241f61e32 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.jmh.record; +import kafka.log.UnifiedLog; import kafka.server.BrokerTopicStats; import kafka.server.RequestLocal; import org.apache.kafka.common.header.Header; @@ -26,6 +27,7 @@ import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.server.log.internals.LogValidator; import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; @@ -75,7 +77,8 @@ public abstract class BaseRecordBatchBenchmark { // Used by measureVariableBatchSize ByteBuffer[] batchBuffers; RequestLocal requestLocal; - final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); + LogValidator.MetricsRecorder validatorMetricsRecorder = UnifiedLog.newValidatorMetricsRecorder( + new BrokerTopicStats().allTopicsStats()); @Setup public void init() { diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java index dd21c19b7a5..7abbfd886f0 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java @@ -16,15 +16,15 @@ */ package org.apache.kafka.jmh.record; -import kafka.common.LongRef; -import kafka.log.AppendOrigin; -import kafka.log.LogValidator; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.utils.PrimitiveRef; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.AppendOrigin; +import org.apache.kafka.server.log.internals.LogValidator; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; @@ -51,14 +51,11 @@ public class CompressedRecordBatchValidationBenchmark extends BaseRecordBatchBen @Benchmark public void measureValidateMessagesAndAssignOffsetsCompressed(Blackhole bh) { MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate()); - LogValidator.validateMessagesAndAssignOffsetsCompressed(records, new TopicPartition("a", 0), - new LongRef(startingOffset), Time.SYSTEM, System.currentTimeMillis(), - compressionType, - compressionType, - false, messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, - new AppendOrigin.Client$(), - MetadataVersion.latest(), - brokerTopicStats, - requestLocal); + new LogValidator(records, new TopicPartition("a", 0), + Time.SYSTEM, compressionType, compressionType, false, messageVersion, + TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT, + MetadataVersion.latest() + ).validateMessagesAndAssignOffsetsCompressed(PrimitiveRef.ofLong(startingOffset), + validatorMetricsRecorder, requestLocal.bufferSupplier()); } } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java index 001837ef6ee..f100a63ec4d 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java @@ -16,13 +16,15 @@ */ package org.apache.kafka.jmh.record; -import kafka.common.LongRef; -import kafka.log.AppendOrigin; -import kafka.log.LogValidator; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.utils.PrimitiveRef; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.AppendOrigin; +import org.apache.kafka.server.log.internals.LogValidator; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; @@ -45,9 +47,10 @@ public class UncompressedRecordBatchValidationBenchmark extends BaseRecordBatchB @Benchmark public void measureAssignOffsetsNonCompressed(Blackhole bh) { MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate()); - LogValidator.assignOffsetsNonCompressed(records, new TopicPartition("a", 0), - new LongRef(startingOffset), System.currentTimeMillis(), false, - TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, - new AppendOrigin.Client$(), messageVersion, brokerTopicStats); + new LogValidator(records, new TopicPartition("a", 0), + Time.SYSTEM, CompressionType.NONE, CompressionType.NONE, false, + messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT, + MetadataVersion.latest() + ).assignOffsetsNonCompressed(PrimitiveRef.ofLong(startingOffset), validatorMetricsRecorder); } } diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java b/storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java new file mode 100644 index 00000000000..aba8faa89fd --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.internals; + +/** + * The source of an append to the log. This is used when determining required validations. + */ +public enum AppendOrigin { + /** + * The log append came through replication from the leader. This typically implies minimal validation. + * Particularly, we do not decompress record batches in order to validate records individually. + */ + REPLICATION, + + /** + * The log append came from either the group coordinator or the transaction coordinator. We validate + * producer epochs for normal log entries (specifically offset commits from the group coordinator) and + * we validate coordinate end transaction markers from the transaction coordinator. + */ + COORDINATOR, + + /** + * The log append came from the client, which implies full validation. + */ + CLIENT, + + /** + * The log append come from the raft leader, which implies the offsets has been assigned + */ + RAFT_LEADER; +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java b/storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java new file mode 100644 index 00000000000..ef693b59a26 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java @@ -0,0 +1,631 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.internals; + +import static org.apache.kafka.server.common.MetadataVersion.IBP_2_1_IV0; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.CorruptRecordException; +import org.apache.kafka.common.errors.InvalidTimestampException; +import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; +import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.AbstractRecords; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.MemoryRecordsBuilder.RecordsInfo; +import org.apache.kafka.common.record.MutableRecordBatch; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.RecordConversionStats; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.requests.ProduceResponse.RecordError; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.CloseableIterator; +import org.apache.kafka.common.utils.PrimitiveRef; +import org.apache.kafka.common.utils.PrimitiveRef.LongRef; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.common.MetadataVersion; + +import java.util.Iterator; + +public class LogValidator { + + public interface MetricsRecorder { + void recordInvalidMagic(); + + void recordInvalidOffset(); + + void recordInvalidSequence(); + + void recordInvalidChecksums(); + + void recordNoKeyCompactedTopic(); + } + + public static class ValidationResult { + public final long logAppendTimeMs; + public final MemoryRecords validatedRecords; + public final long maxTimestampMs; + public final long shallowOffsetOfMaxTimestampMs; + public final boolean messageSizeMaybeChanged; + public final RecordConversionStats recordConversionStats; + + public ValidationResult(long logAppendTimeMs, MemoryRecords validatedRecords, long maxTimestampMs, + long shallowOffsetOfMaxTimestampMs, boolean messageSizeMaybeChanged, + RecordConversionStats recordConversionStats) { + this.logAppendTimeMs = logAppendTimeMs; + this.validatedRecords = validatedRecords; + this.maxTimestampMs = maxTimestampMs; + this.shallowOffsetOfMaxTimestampMs = shallowOffsetOfMaxTimestampMs; + this.messageSizeMaybeChanged = messageSizeMaybeChanged; + this.recordConversionStats = recordConversionStats; + } + } + + private static class ApiRecordError { + final Errors apiError; + final RecordError recordError; + + private ApiRecordError(Errors apiError, RecordError recordError) { + this.apiError = apiError; + this.recordError = recordError; + } + } + + private final MemoryRecords records; + private final TopicPartition topicPartition; + private final Time time; + private final CompressionType sourceCompression; + private final CompressionType targetCompression; + private final boolean compactedTopic; + private final byte toMagic; + private final TimestampType timestampType; + private final long timestampDiffMaxMs; + private final int partitionLeaderEpoch; + private final AppendOrigin origin; + private final MetadataVersion interBrokerProtocolVersion; + + public LogValidator(MemoryRecords records, + TopicPartition topicPartition, + Time time, + CompressionType sourceCompression, + CompressionType targetCompression, + boolean compactedTopic, + byte toMagic, + TimestampType timestampType, + long timestampDiffMaxMs, + int partitionLeaderEpoch, + AppendOrigin origin, + MetadataVersion interBrokerProtocolVersion) { + this.records = records; + this.topicPartition = topicPartition; + this.time = time; + this.sourceCompression = sourceCompression; + this.targetCompression = targetCompression; + this.compactedTopic = compactedTopic; + this.toMagic = toMagic; + this.timestampType = timestampType; + this.timestampDiffMaxMs = timestampDiffMaxMs; + this.partitionLeaderEpoch = partitionLeaderEpoch; + this.origin = origin; + this.interBrokerProtocolVersion = interBrokerProtocolVersion; + } + + /** + * Update the offsets for this message set and do further validation on messages including: + * 1. Messages for compacted topics must have keys + * 2. When magic value >= 1, inner messages of a compressed message set must have monotonically increasing offsets + * starting from 0. + * 3. When magic value >= 1, validate and maybe overwrite timestamps of messages. + * 4. Declared count of records in DefaultRecordBatch must match number of valid records contained therein. + * + * This method will convert messages as necessary to the topic's configured message format version. If no format + * conversion or value overwriting is required for messages, this method will perform in-place operations to + * avoid expensive re-compression. + * + * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset + * 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, + BufferSupplier bufferSupplier) { + if (sourceCompression == CompressionType.NONE && targetCompression == CompressionType.NONE) { + // check the magic value + if (!records.hasMatchingMagic(toMagic)) + return convertAndAssignOffsetsNonCompressed(offsetCounter, metricsRecorder); + else + // Do in-place validation, offset assignment and maybe set timestamp + return assignOffsetsNonCompressed(offsetCounter, metricsRecorder); + } else + return validateMessagesAndAssignOffsetsCompressed(offsetCounter, metricsRecorder, bufferSupplier); + + } + + private static MutableRecordBatch getFirstBatchAndMaybeValidateNoMoreBatches(MemoryRecords records, + CompressionType sourceCompression) { + Iterator batchIterator = records.batches().iterator(); + + if (!batchIterator.hasNext()) + throw new InvalidRecordException("Record batch has no batches at all"); + + MutableRecordBatch batch = batchIterator.next(); + + // if the format is v2 and beyond, or if the messages are compressed, we should check there's only one batch. + if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2 || sourceCompression != CompressionType.NONE) { + if (batchIterator.hasNext()) + throw new InvalidRecordException("Compressed outer record has more than one batch"); + } + + return batch; + } + + private ValidationResult convertAndAssignOffsetsNonCompressed(LongRef offsetCounter, + MetricsRecorder metricsRecorder) { + long now = time.milliseconds(); + long startNanos = time.nanoseconds(); + int sizeInBytesAfterConversion = AbstractRecords.estimateSizeInBytes(toMagic, offsetCounter.value, + CompressionType.NONE, records.records()); + + RecordBatch firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE); + long producerId = firstBatch.producerId(); + short producerEpoch = firstBatch.producerEpoch(); + int sequence = firstBatch.baseSequence(); + boolean isTransactional = firstBatch.isTransactional(); + + // The current implementation of BufferSupplier is naive and works best when the buffer size + // cardinality is low, so don't use it here + ByteBuffer newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion); + MemoryRecordsBuilder builder = MemoryRecords.builder(newBuffer, toMagic, CompressionType.NONE, + timestampType, offsetCounter.value, now, producerId, producerEpoch, sequence, isTransactional, + partitionLeaderEpoch); + + for (RecordBatch batch : records.batches()) { + validateBatch(topicPartition, firstBatch, batch, origin, toMagic, metricsRecorder); + + List recordErrors = new ArrayList<>(0); + int batchIndex = 0; + for (Record record : batch) { + Optional recordError = validateRecord(batch, topicPartition, + record, batchIndex, now, timestampType, timestampDiffMaxMs, compactedTopic, + metricsRecorder); + recordError.ifPresent(e -> recordErrors.add(e)); + // we fail the batch if any record fails, so we stop appending if any record fails + if (recordErrors.isEmpty()) + builder.appendWithOffset(offsetCounter.value++, record); + ++batchIndex; + } + + processRecordErrors(recordErrors); + } + + MemoryRecords convertedRecords = builder.build(); + + RecordsInfo info = builder.info(); + RecordConversionStats recordConversionStats = new RecordConversionStats( + builder.uncompressedBytesWritten(), builder.numRecords(), time.nanoseconds() - startNanos); + return new ValidationResult( + now, + convertedRecords, + info.maxTimestamp, + info.shallowOffsetOfMaxTimestamp, + true, + recordConversionStats); + } + + // Visible for benchmarking + public ValidationResult assignOffsetsNonCompressed(LongRef offsetCounter, + MetricsRecorder metricsRecorder) { + long now = time.milliseconds(); + long maxTimestamp = RecordBatch.NO_TIMESTAMP; + long offsetOfMaxTimestamp = -1L; + long initialOffset = offsetCounter.value; + + RecordBatch firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE); + + for (MutableRecordBatch batch : records.batches()) { + validateBatch(topicPartition, firstBatch, batch, origin, toMagic, metricsRecorder); + + long maxBatchTimestamp = RecordBatch.NO_TIMESTAMP; + long offsetOfMaxBatchTimestamp = -1L; + + List recordErrors = new ArrayList<>(0); + // This is a hot path and we want to avoid any unnecessary allocations. + // That said, there is no benefit in using `skipKeyValueIterator` for the uncompressed + // case since we don't do key/value copies in this path (we just slice the ByteBuffer) + int batchIndex = 0; + for (Record record : batch) { + Optional recordError = validateRecord(batch, topicPartition, record, + batchIndex, now, timestampType, timestampDiffMaxMs, compactedTopic, metricsRecorder); + recordError.ifPresent(e -> recordErrors.add(e)); + + long offset = offsetCounter.value++; + if (batch.magic() > RecordBatch.MAGIC_VALUE_V0 && record.timestamp() > maxBatchTimestamp) { + maxBatchTimestamp = record.timestamp(); + offsetOfMaxBatchTimestamp = offset; + } + ++batchIndex; + } + + processRecordErrors(recordErrors); + + if (batch.magic() > RecordBatch.MAGIC_VALUE_V0 && maxBatchTimestamp > maxTimestamp) { + maxTimestamp = maxBatchTimestamp; + offsetOfMaxTimestamp = offsetOfMaxBatchTimestamp; + } + + batch.setLastOffset(offsetCounter.value - 1); + + if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2) + batch.setPartitionLeaderEpoch(partitionLeaderEpoch); + + if (batch.magic() > RecordBatch.MAGIC_VALUE_V0) { + if (timestampType == TimestampType.LOG_APPEND_TIME) + batch.setMaxTimestamp(TimestampType.LOG_APPEND_TIME, now); + else + batch.setMaxTimestamp(timestampType, maxBatchTimestamp); + } + } + + if (timestampType == TimestampType.LOG_APPEND_TIME) { + maxTimestamp = now; + if (toMagic >= RecordBatch.MAGIC_VALUE_V2) + offsetOfMaxTimestamp = offsetCounter.value - 1; + else + offsetOfMaxTimestamp = initialOffset; + } + + return new ValidationResult( + now, + records, + maxTimestamp, + offsetOfMaxTimestamp, + false, + RecordConversionStats.EMPTY); + } + + /** + * We cannot do in place assignment in one of the following situations: + * 1. Source and target compression codec are different + * 2. When the target magic is not equal to batches' magic, meaning format conversion is needed. + * 3. When the target magic is equal to V0, meaning absolute offsets need to be re-assigned. + */ + // Visible for benchmarking + public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offsetCounter, + MetricsRecorder metricsRecorder, + BufferSupplier bufferSupplier) { + if (targetCompression == CompressionType.ZSTD && interBrokerProtocolVersion.isLessThan(IBP_2_1_IV0)) + throw new UnsupportedCompressionTypeException("Produce requests to inter.broker.protocol.version < 2.1 broker " + + "are not allowed to use ZStandard compression"); + + // No in place assignment situation 1 + boolean inPlaceAssignment = sourceCompression == targetCompression; + long now = time.milliseconds(); + + long maxTimestamp = RecordBatch.NO_TIMESTAMP; + LongRef expectedInnerOffset = PrimitiveRef.ofLong(0); + List validatedRecords = new ArrayList<>(); + + int uncompressedSizeInBytes = 0; + + // Assume there's only one batch with compressed memory records; otherwise, return InvalidRecordException + // One exception though is that with format smaller than v2, if sourceCompression is noCompression, then each batch is actually + // a single record so we'd need to special handle it by creating a single wrapper batch that includes all the records + MutableRecordBatch firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, sourceCompression); + + // No in place assignment situation 2 and 3: we only need to check for the first batch because: + // 1. For most cases (compressed records, v2, for example), there's only one batch anyways. + // 2. For cases that there may be multiple batches, all batches' magic should be the same. + if (firstBatch.magic() != toMagic || toMagic == RecordBatch.MAGIC_VALUE_V0) + inPlaceAssignment = false; + + // Do not compress control records unless they are written compressed + if (sourceCompression == CompressionType.NONE && firstBatch.isControlBatch()) + inPlaceAssignment = true; + + for (MutableRecordBatch batch : records.batches()) { + validateBatch(topicPartition, firstBatch, batch, origin, toMagic, metricsRecorder); + uncompressedSizeInBytes += AbstractRecords.recordBatchHeaderSizeInBytes(toMagic, batch.compressionType()); + + // if we are on version 2 and beyond, and we know we are going for in place assignment, + // then we can optimize the iterator to skip key / value / headers since they would not be used at all + CloseableIterator recordsIterator; + if (inPlaceAssignment && firstBatch.magic() >= RecordBatch.MAGIC_VALUE_V2) + recordsIterator = batch.skipKeyValueIterator(bufferSupplier); + else + recordsIterator = batch.streamingIterator(bufferSupplier); + + try { + List recordErrors = new ArrayList<>(0); + // this is a hot path and we want to avoid any unnecessary allocations. + int batchIndex = 0; + while (recordsIterator.hasNext()) { + Record record = recordsIterator.next(); + long expectedOffset = expectedInnerOffset.value++; + + Optional recordError = validateRecordCompression(sourceCompression, + batchIndex, record); + if (!recordError.isPresent()) { + recordError = validateRecord(batch, topicPartition, record, batchIndex, now, + timestampType, timestampDiffMaxMs, compactedTopic, metricsRecorder); + } + + if (!recordError.isPresent() + && batch.magic() > RecordBatch.MAGIC_VALUE_V0 + && toMagic > RecordBatch.MAGIC_VALUE_V0) { + + if (record.timestamp() > maxTimestamp) + maxTimestamp = record.timestamp(); + + // Some older clients do not implement the V1 internal offsets correctly. + // Historically the broker handled this by rewriting the batches rather + // than rejecting the request. We must continue this handling here to avoid + // breaking these clients. + if (record.offset() != expectedOffset) + inPlaceAssignment = false; + } + + if (recordError.isPresent()) + recordErrors.add(recordError.get()); + else { + uncompressedSizeInBytes += record.sizeInBytes(); + validatedRecords.add(record); + } + + ++batchIndex; + } + + processRecordErrors(recordErrors); + + } finally { + recordsIterator.close(); + } + } + + if (!inPlaceAssignment) { + return buildRecordsAndAssignOffsets(offsetCounter, now, firstBatch, validatedRecords, + uncompressedSizeInBytes); + } else { + // we can update the batch only and write the compressed payload as is; + // again we assume only one record batch within the compressed set + offsetCounter.value += validatedRecords.size(); + long lastOffset = offsetCounter.value - 1; + firstBatch.setLastOffset(lastOffset); + + if (timestampType == TimestampType.LOG_APPEND_TIME) + maxTimestamp = now; + + if (toMagic >= RecordBatch.MAGIC_VALUE_V1) + firstBatch.setMaxTimestamp(timestampType, maxTimestamp); + + if (toMagic >= RecordBatch.MAGIC_VALUE_V2) + firstBatch.setPartitionLeaderEpoch(partitionLeaderEpoch); + + RecordConversionStats recordConversionStats = new RecordConversionStats(uncompressedSizeInBytes, 0, 0); + return new ValidationResult( + now, + records, + maxTimestamp, + lastOffset, + false, + recordConversionStats); + } + } + + private ValidationResult buildRecordsAndAssignOffsets(LongRef offsetCounter, + long logAppendTime, + RecordBatch firstBatch, + List validatedRecords, + int uncompressedSizeInBytes) { + long startNanos = time.nanoseconds(); + int estimatedSize = AbstractRecords.estimateSizeInBytes(toMagic, offsetCounter.value, targetCompression, + validatedRecords); + // The current implementation of BufferSupplier is naive and works best when the buffer size + // cardinality is low, so don't use it here + ByteBuffer buffer = ByteBuffer.allocate(estimatedSize); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, toMagic, targetCompression, + timestampType, offsetCounter.value, logAppendTime, firstBatch.producerId(), + firstBatch.producerEpoch(), firstBatch.baseSequence(), firstBatch.isTransactional(), + partitionLeaderEpoch); + + for (Record record : validatedRecords) + builder.appendWithOffset(offsetCounter.value++, record); + + MemoryRecords records = builder.build(); + + RecordsInfo info = builder.info(); + + // This is not strictly correct, it represents the number of records where in-place assignment is not possible + // instead of the number of records that were converted. It will over-count cases where the source and target are + // message format V0 or if the inner offsets are not consecutive. This is OK since the impact is the same: we have + // to rebuild the records (including recompression if enabled). + int conversionCount = builder.numRecords(); + RecordConversionStats recordConversionStats = new RecordConversionStats( + uncompressedSizeInBytes + builder.uncompressedBytesWritten(), conversionCount, + time.nanoseconds() - startNanos); + + return new ValidationResult( + logAppendTime, + records, + info.maxTimestamp, + info.shallowOffsetOfMaxTimestamp, + true, + recordConversionStats); + } + + + private static void validateBatch(TopicPartition topicPartition, + RecordBatch firstBatch, + RecordBatch batch, + AppendOrigin origin, + byte toMagic, + MetricsRecorder metricsRecorder) { + // batch magic byte should have the same magic as the first batch + if (firstBatch.magic() != batch.magic()) { + metricsRecorder.recordInvalidMagic(); + throw new InvalidRecordException("Batch magic " + batch.magic() + " is not the same as the first batch's magic byte " + + firstBatch.magic() + " in topic partition " + topicPartition); + } + + if (origin == AppendOrigin.CLIENT) { + if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2) { + long countFromOffsets = batch.lastOffset() - batch.baseOffset() + 1; + if (countFromOffsets <= 0) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Batch has an invalid offset range: [" + batch.baseOffset() + ", " + + batch.lastOffset() + "] in topic partition " + topicPartition); + } + + // v2 and above messages always have a non-null count + long count = batch.countOrNull(); + if (count <= 0) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Invalid reported count for record batch: " + count + + " in topic partition " + topicPartition); + } + + if (countFromOffsets != count) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Inconsistent batch offset range [" + batch.baseOffset() + ", " + + batch.lastOffset() + "] and count of records " + count + " in topic partition " + topicPartition); + } + } + + if (batch.isControlBatch()) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Clients are not allowed to write control records in topic partition " + topicPartition); + } + + if (batch.hasProducerId() && batch.baseSequence() < 0) { + metricsRecorder.recordInvalidSequence(); + throw new InvalidRecordException("Invalid sequence number " + batch.baseSequence() + " in record batch with producerId " + + batch.producerId() + " in topic partition " + topicPartition); + } + } + + if (batch.isTransactional() && toMagic < RecordBatch.MAGIC_VALUE_V2) + throw new UnsupportedForMessageFormatException("Transactional records cannot be used with magic version " + toMagic); + + if (batch.hasProducerId() && toMagic < RecordBatch.MAGIC_VALUE_V2) + throw new UnsupportedForMessageFormatException("Idempotent records cannot be used with magic version " + toMagic); + } + + private static Optional validateRecord(RecordBatch batch, + TopicPartition topicPartition, + Record record, + int batchIndex, + long now, + TimestampType timestampType, + long timestampDiffMaxMs, + boolean compactedTopic, + MetricsRecorder metricsRecorder) { + if (!record.hasMagic(batch.magic())) { + metricsRecorder.recordInvalidMagic(); + return Optional.of(new ApiRecordError(Errors.INVALID_RECORD, + new RecordError(batchIndex, "Record " + record + + "'s magic does not match outer magic " + batch.magic() + " in topic partition " + + topicPartition))); + } + + // verify the record-level CRC only if this is one of the deep entries of a compressed message + // set for magic v0 and v1. For non-compressed messages, there is no inner record for magic v0 and v1, + // so we depend on the batch-level CRC check in Log.analyzeAndValidateRecords(). For magic v2 and above, + // there is no record-level CRC to check. + if (batch.magic() <= RecordBatch.MAGIC_VALUE_V1 && batch.isCompressed()) { + try { + record.ensureValid(); + } catch (InvalidRecordException e) { + metricsRecorder.recordInvalidChecksums(); + throw new CorruptRecordException(e.getMessage() + " in topic partition " + topicPartition); + } + } + + Optional keyError = validateKey(record, batchIndex, topicPartition, + compactedTopic, metricsRecorder); + if (keyError.isPresent()) + return keyError; + else + return validateTimestamp(batch, record, batchIndex, now, timestampType, timestampDiffMaxMs); + } + + private static Optional validateKey(Record record, + int batchIndex, + TopicPartition topicPartition, + boolean compactedTopic, + MetricsRecorder metricsRecorder) { + if (compactedTopic && !record.hasKey()) { + metricsRecorder.recordNoKeyCompactedTopic(); + return Optional.of(new ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, + "Compacted topic cannot accept message without key in topic partition " + + topicPartition))); + } else + return Optional.empty(); + } + + private static Optional validateTimestamp(RecordBatch batch, + Record record, + int batchIndex, + long now, + TimestampType timestampType, + long timestampDiffMaxMs) { + if (timestampType == TimestampType.CREATE_TIME + && record.timestamp() != RecordBatch.NO_TIMESTAMP + && Math.abs(record.timestamp() - now) > timestampDiffMaxMs) + return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, + "Timestamp " + record.timestamp() + " of message with offset " + record.offset() + + " is out of range. The timestamp should be within [" + (now - timestampDiffMaxMs) + + ", " + (now + timestampDiffMaxMs) + "]"))); + else if (batch.timestampType() == TimestampType.LOG_APPEND_TIME) + return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, + "Invalid timestamp type in message " + record + ". Producer should not set timestamp " + + "type to LogAppendTime."))); + else + return Optional.empty(); + } + + private static Optional validateRecordCompression(CompressionType sourceCompression, + int batchIndex, + Record record) { + if (sourceCompression != CompressionType.NONE && record.isCompressed()) + return Optional.of(new ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, + "Compressed outer record should not have an inner record with a compression attribute set: " + + record))); + else + return Optional.empty(); + } + + private static void processRecordErrors(List recordErrors) { + if (!recordErrors.isEmpty()) { + List errors = recordErrors.stream().map(e -> e.recordError).collect(Collectors.toList()); + if (recordErrors.stream().anyMatch(e -> e.apiError == Errors.INVALID_TIMESTAMP)) { + throw new RecordValidationException(new InvalidTimestampException( + "One or more records have been rejected due to invalid timestamp"), errors); + } else { + throw new RecordValidationException(new InvalidRecordException( + "One or more records have been rejected due to " + errors.size() + " record errors " + + "in total, and only showing the first three errors at most: " + errors.subList(0, Math.min(errors.size(), 3))), errors); + } + } + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java b/storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java new file mode 100644 index 00000000000..164e1e209c6 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.internals; + +import java.util.Collections; +import java.util.List; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.requests.ProduceResponse.RecordError; + +public class RecordValidationException extends RuntimeException { + private final ApiException invalidException; + private final List recordErrors; + + public RecordValidationException(ApiException invalidException, List recordErrors) { + super(invalidException); + this.invalidException = invalidException; + this.recordErrors = Collections.unmodifiableList(recordErrors); + } + + public ApiException invalidException() { + return invalidException; + } + + public List recordErrors() { + return recordErrors; + } +}