mirror of https://github.com/apache/kafka.git
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 <junrao@gmail.com>
This commit is contained in:
parent
ca15735fa7
commit
e8232edd24
|
@ -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
|
||||
|
|
|
@ -318,9 +318,13 @@
|
|||
<suppress checks="AvoidStarImport"
|
||||
files="MetadataVersionTest.java"/>
|
||||
|
||||
<!-- Storage -->
|
||||
<suppress checks="(CyclomaticComplexity|ParameterNumber)"
|
||||
files="(RemoteLogManagerConfig).java"/>
|
||||
<!-- storage -->
|
||||
<suppress checks="CyclomaticComplexity"
|
||||
files="(LogValidator|RemoteLogManagerConfig).java"/>
|
||||
<suppress checks="NPathComplexity"
|
||||
files="LogValidator.java"/>
|
||||
<suppress checks="ParameterNumber"
|
||||
files="RemoteLogManagerConfig.java"/>
|
||||
|
||||
<!-- benchmarks -->
|
||||
<suppress checks="(ClassDataAbstractionCoupling|ClassFanOutComplexity)"
|
||||
|
|
|
@ -26,11 +26,34 @@ public class PrimitiveRef {
|
|||
return new IntRef(value);
|
||||
}
|
||||
|
||||
public static LongRef ofLong(long value) {
|
||||
return new LongRef(value);
|
||||
}
|
||||
|
||||
public static class IntRef {
|
||||
public int value;
|
||||
|
||||
IntRef(int value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "IntRef(" + value + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class LongRef {
|
||||
public long value;
|
||||
|
||||
LongRef(long value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LongRef(" + value + ")";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* 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.common.utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class PrimitiveRefTest {
|
||||
|
||||
@Test
|
||||
public void testIntRef() {
|
||||
PrimitiveRef.IntRef ref = PrimitiveRef.ofInt(3);
|
||||
assertEquals(3, ref.value++);
|
||||
assertEquals(4, ref.value);
|
||||
assertEquals(5, ++ref.value);
|
||||
assertEquals(5, ref.value);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLongRef() {
|
||||
PrimitiveRef.LongRef ref = PrimitiveRef.ofLong(5L);
|
||||
assertEquals(5L, ref.value++);
|
||||
assertEquals(6L, ref.value);
|
||||
assertEquals(7L, ++ref.value);
|
||||
assertEquals(7L, ref.value);
|
||||
}
|
||||
|
||||
}
|
|
@ -44,6 +44,7 @@ import org.apache.kafka.common.utils.Time
|
|||
import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.log.internals.AppendOrigin
|
||||
|
||||
import scala.collection.{Map, Seq}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
|
|
@ -1,28 +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.common
|
||||
|
||||
import org.apache.kafka.common.errors.ApiException
|
||||
import org.apache.kafka.common.requests.ProduceResponse.RecordError
|
||||
|
||||
import scala.collection.Seq
|
||||
|
||||
class RecordValidationException(val invalidException: ApiException,
|
||||
val recordErrors: Seq[RecordError])
|
||||
extends RuntimeException(invalidException) {
|
||||
}
|
|
@ -24,11 +24,9 @@ import java.util.Optional
|
|||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.common.OffsetAndMetadata
|
||||
import kafka.internals.generated.{GroupMetadataValue, OffsetCommitKey, OffsetCommitValue, GroupMetadataKey => 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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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}")
|
||||
|
|
|
@ -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 += _)
|
||||
}
|
||||
}
|
||||
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
)
|
||||
)
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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)))
|
||||
|
|
|
@ -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 = {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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}
|
||||
|
||||
|
|
|
@ -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]],
|
||||
|
|
|
@ -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]],
|
||||
|
|
|
@ -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]],
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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] => {
|
||||
|
|
|
@ -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,
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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
|
||||
)
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -495,6 +495,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
|
|||
<Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
|
||||
</Match>
|
||||
|
||||
<Match>
|
||||
<!-- False positive - the field is used in other modules and its unit test -->
|
||||
<Class name="org.apache.kafka.common.utils.PrimitiveRef$LongRef"/>
|
||||
<Field name="value"/>
|
||||
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD"/>
|
||||
</Match>
|
||||
|
||||
<Match>
|
||||
<!-- Suppress warnings related to jmh generated code -->
|
||||
<Package name="org.apache.kafka.jmh.acl.generated"/>
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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<MutableRecordBatch> 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<ApiRecordError> recordErrors = new ArrayList<>(0);
|
||||
int batchIndex = 0;
|
||||
for (Record record : batch) {
|
||||
Optional<ApiRecordError> 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<ApiRecordError> 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<ApiRecordError> 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<Record> 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<Record> recordsIterator;
|
||||
if (inPlaceAssignment && firstBatch.magic() >= RecordBatch.MAGIC_VALUE_V2)
|
||||
recordsIterator = batch.skipKeyValueIterator(bufferSupplier);
|
||||
else
|
||||
recordsIterator = batch.streamingIterator(bufferSupplier);
|
||||
|
||||
try {
|
||||
List<ApiRecordError> 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<ApiRecordError> 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<Record> 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<ApiRecordError> 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<ApiRecordError> keyError = validateKey(record, batchIndex, topicPartition,
|
||||
compactedTopic, metricsRecorder);
|
||||
if (keyError.isPresent())
|
||||
return keyError;
|
||||
else
|
||||
return validateTimestamp(batch, record, batchIndex, now, timestampType, timestampDiffMaxMs);
|
||||
}
|
||||
|
||||
private static Optional<ApiRecordError> 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<ApiRecordError> 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<ApiRecordError> 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<ApiRecordError> recordErrors) {
|
||||
if (!recordErrors.isEmpty()) {
|
||||
List<RecordError> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<RecordError> recordErrors;
|
||||
|
||||
public RecordValidationException(ApiException invalidException, List<RecordError> recordErrors) {
|
||||
super(invalidException);
|
||||
this.invalidException = invalidException;
|
||||
this.recordErrors = Collections.unmodifiableList(recordErrors);
|
||||
}
|
||||
|
||||
public ApiException invalidException() {
|
||||
return invalidException;
|
||||
}
|
||||
|
||||
public List<RecordError> recordErrors() {
|
||||
return recordErrors;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue