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(':server-common')
|
||||||
implementation project(':clients')
|
implementation project(':clients')
|
||||||
implementation project(':metadata')
|
implementation project(':metadata')
|
||||||
|
implementation project(':storage')
|
||||||
implementation project(':streams')
|
implementation project(':streams')
|
||||||
implementation project(':core')
|
implementation project(':core')
|
||||||
implementation project(':clients').sourceSets.test.output
|
implementation project(':clients').sourceSets.test.output
|
||||||
|
|
|
@ -318,9 +318,13 @@
|
||||||
<suppress checks="AvoidStarImport"
|
<suppress checks="AvoidStarImport"
|
||||||
files="MetadataVersionTest.java"/>
|
files="MetadataVersionTest.java"/>
|
||||||
|
|
||||||
<!-- Storage -->
|
<!-- storage -->
|
||||||
<suppress checks="(CyclomaticComplexity|ParameterNumber)"
|
<suppress checks="CyclomaticComplexity"
|
||||||
files="(RemoteLogManagerConfig).java"/>
|
files="(LogValidator|RemoteLogManagerConfig).java"/>
|
||||||
|
<suppress checks="NPathComplexity"
|
||||||
|
files="LogValidator.java"/>
|
||||||
|
<suppress checks="ParameterNumber"
|
||||||
|
files="RemoteLogManagerConfig.java"/>
|
||||||
|
|
||||||
<!-- benchmarks -->
|
<!-- benchmarks -->
|
||||||
<suppress checks="(ClassDataAbstractionCoupling|ClassFanOutComplexity)"
|
<suppress checks="(ClassDataAbstractionCoupling|ClassFanOutComplexity)"
|
||||||
|
|
|
@ -26,11 +26,34 @@ public class PrimitiveRef {
|
||||||
return new IntRef(value);
|
return new IntRef(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static LongRef ofLong(long value) {
|
||||||
|
return new LongRef(value);
|
||||||
|
}
|
||||||
|
|
||||||
public static class IntRef {
|
public static class IntRef {
|
||||||
public int value;
|
public int value;
|
||||||
|
|
||||||
IntRef(int value) {
|
IntRef(int value) {
|
||||||
this.value = 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.common.{IsolationLevel, TopicPartition, Uuid}
|
||||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
import org.apache.kafka.server.common.MetadataVersion
|
||||||
|
import org.apache.kafka.server.log.internals.AppendOrigin
|
||||||
|
|
||||||
import scala.collection.{Map, Seq}
|
import scala.collection.{Map, Seq}
|
||||||
import scala.jdk.CollectionConverters._
|
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.atomic.AtomicBoolean
|
||||||
import java.util.concurrent.locks.ReentrantLock
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
|
import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
|
||||||
|
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
import kafka.common.OffsetAndMetadata
|
import kafka.common.OffsetAndMetadata
|
||||||
import kafka.internals.generated.{GroupMetadataValue, OffsetCommitKey, OffsetCommitValue, GroupMetadataKey => GroupMetadataKeyData}
|
import kafka.internals.generated.{GroupMetadataValue, OffsetCommitKey, OffsetCommitValue, GroupMetadataKey => GroupMetadataKeyData}
|
||||||
import kafka.log.AppendOrigin
|
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.server.{FetchLogEnd, ReplicaManager, RequestLocal}
|
import kafka.server.{FetchLogEnd, ReplicaManager, RequestLocal}
|
||||||
import kafka.utils.CoreUtils.inLock
|
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.common.{KafkaException, MessageFormatter, TopicPartition}
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
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.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._
|
||||||
import scala.collection.mutable.ArrayBuffer
|
import scala.collection.mutable.ArrayBuffer
|
||||||
|
@ -331,7 +330,7 @@ class GroupMetadataManager(brokerId: Int,
|
||||||
timeout = config.offsetCommitTimeoutMs.toLong,
|
timeout = config.offsetCommitTimeoutMs.toLong,
|
||||||
requiredAcks = config.offsetCommitRequiredAcks,
|
requiredAcks = config.offsetCommitRequiredAcks,
|
||||||
internalTopicsAllowed = true,
|
internalTopicsAllowed = true,
|
||||||
origin = AppendOrigin.Coordinator,
|
origin = AppendOrigin.COORDINATOR,
|
||||||
entriesPerPartition = records,
|
entriesPerPartition = records,
|
||||||
delayedProduceLock = Some(group.lock),
|
delayedProduceLock = Some(group.lock),
|
||||||
responseCallback = callback,
|
responseCallback = callback,
|
||||||
|
@ -890,7 +889,7 @@ class GroupMetadataManager(brokerId: Int,
|
||||||
// do not need to require acks since even if the tombstone is lost,
|
// do not need to require acks since even if the tombstone is lost,
|
||||||
// it will be appended again in the next purge cycle
|
// it will be appended again in the next purge cycle
|
||||||
val records = MemoryRecords.withRecords(magicValue, 0L, compressionType, timestampType, tombstones.toArray: _*)
|
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)
|
requestLocal = requestLocal)
|
||||||
|
|
||||||
offsetsRemoved += removedOffsets.size
|
offsetsRemoved += removedOffsets.size
|
||||||
|
|
|
@ -16,12 +16,13 @@
|
||||||
*/
|
*/
|
||||||
package kafka.coordinator.transaction
|
package kafka.coordinator.transaction
|
||||||
|
|
||||||
|
import kafka.log.LogConfig
|
||||||
|
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||||
import kafka.log.{AppendOrigin, LogConfig}
|
|
||||||
import kafka.server.{Defaults, FetchLogEnd, ReplicaManager, RequestLocal}
|
import kafka.server.{Defaults, FetchLogEnd, ReplicaManager, RequestLocal}
|
||||||
import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
|
import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
|
||||||
import kafka.utils.{Logging, Pool, Scheduler}
|
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.requests.TransactionResult
|
||||||
import org.apache.kafka.common.utils.{Time, Utils}
|
import org.apache.kafka.common.utils.{Time, Utils}
|
||||||
import org.apache.kafka.common.{KafkaException, TopicPartition}
|
import org.apache.kafka.common.{KafkaException, TopicPartition}
|
||||||
|
import org.apache.kafka.server.log.internals.AppendOrigin
|
||||||
import org.apache.kafka.server.record.BrokerCompressionType
|
import org.apache.kafka.server.record.BrokerCompressionType
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -282,7 +284,7 @@ class TransactionStateManager(brokerId: Int,
|
||||||
config.requestTimeoutMs,
|
config.requestTimeoutMs,
|
||||||
TransactionLog.EnforcedRequiredAcks,
|
TransactionLog.EnforcedRequiredAcks,
|
||||||
internalTopicsAllowed = true,
|
internalTopicsAllowed = true,
|
||||||
origin = AppendOrigin.Coordinator,
|
origin = AppendOrigin.COORDINATOR,
|
||||||
entriesPerPartition = Map(transactionPartition -> tombstoneRecords),
|
entriesPerPartition = Map(transactionPartition -> tombstoneRecords),
|
||||||
removeFromCacheCallback,
|
removeFromCacheCallback,
|
||||||
requestLocal = RequestLocal.NoCaching)
|
requestLocal = RequestLocal.NoCaching)
|
||||||
|
@ -761,7 +763,7 @@ class TransactionStateManager(brokerId: Int,
|
||||||
newMetadata.txnTimeoutMs.toLong,
|
newMetadata.txnTimeoutMs.toLong,
|
||||||
TransactionLog.EnforcedRequiredAcks,
|
TransactionLog.EnforcedRequiredAcks,
|
||||||
internalTopicsAllowed = true,
|
internalTopicsAllowed = true,
|
||||||
origin = AppendOrigin.Coordinator,
|
origin = AppendOrigin.COORDINATOR,
|
||||||
recordsPerPartition,
|
recordsPerPartition,
|
||||||
updateCacheCallback,
|
updateCacheCallback,
|
||||||
requestLocal = requestLocal)
|
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.FileRecords.{LogOffsetPosition, TimestampAndOffset}
|
||||||
import org.apache.kafka.common.record._
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.{BufferSupplier, Time}
|
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 java.util.Optional
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -248,7 +248,7 @@ class LogSegment private[log] (val log: FileRecords,
|
||||||
private def updateProducerState(producerStateManager: ProducerStateManager, batch: RecordBatch): Unit = {
|
private def updateProducerState(producerStateManager: ProducerStateManager, batch: RecordBatch): Unit = {
|
||||||
if (batch.hasProducerId) {
|
if (batch.hasProducerId) {
|
||||||
val producerId = batch.producerId
|
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)
|
val maybeCompletedTxn = appendInfo.append(batch, firstOffsetMetadataOpt = None)
|
||||||
producerStateManager.update(appendInfo)
|
producerStateManager.update(appendInfo)
|
||||||
maybeCompletedTxn.foreach { completedTxn =>
|
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.protocol.types._
|
||||||
import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, EndTransactionMarker, RecordBatch}
|
import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, EndTransactionMarker, RecordBatch}
|
||||||
import org.apache.kafka.common.utils.{ByteUtils, Crc32C, Time, Utils}
|
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.jdk.CollectionConverters._
|
||||||
import scala.collection.mutable.ListBuffer
|
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 = {
|
private def maybeValidateDataBatch(producerEpoch: Short, firstSeq: Int, offset: Long): Unit = {
|
||||||
checkProducerEpoch(producerEpoch, offset)
|
checkProducerEpoch(producerEpoch, offset)
|
||||||
if (origin == AppendOrigin.Client) {
|
if (origin == AppendOrigin.CLIENT) {
|
||||||
checkSequence(producerEpoch, firstSeq, offset)
|
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, " +
|
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}"
|
s"which is smaller than the last seen epoch ${updatedEntry.producerEpoch}"
|
||||||
|
|
||||||
if (origin == AppendOrigin.Replication) {
|
if (origin == AppendOrigin.REPLICATION) {
|
||||||
warn(message)
|
warn(message)
|
||||||
} else {
|
} else {
|
||||||
// Starting from 2.7, we replaced ProducerFenced error with InvalidProducerEpoch in the
|
// 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 = {
|
private def checkCoordinatorEpoch(endTxnMarker: EndTransactionMarker, offset: Long): Unit = {
|
||||||
if (updatedEntry.coordinatorEpoch > endTxnMarker.coordinatorEpoch) {
|
if (updatedEntry.coordinatorEpoch > endTxnMarker.coordinatorEpoch) {
|
||||||
if (origin == AppendOrigin.Replication) {
|
if (origin == AppendOrigin.REPLICATION) {
|
||||||
info(s"Detected invalid coordinator epoch for producerId $producerId at " +
|
info(s"Detected invalid coordinator epoch for producerId $producerId at " +
|
||||||
s"offset $offset in partition $topicPartition: ${endTxnMarker.coordinatorEpoch} " +
|
s"offset $offset in partition $topicPartition: ${endTxnMarker.coordinatorEpoch} " +
|
||||||
s"is older than previously known coordinator epoch ${updatedEntry.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.nio.file.Files
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, TimeUnit}
|
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, TimeUnit}
|
||||||
import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException}
|
import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException}
|
||||||
import kafka.log.AppendOrigin.RaftLeader
|
|
||||||
import kafka.log.remote.RemoteLogManager
|
import kafka.log.remote.RemoteLogManager
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.server._
|
|
||||||
import kafka.server.checkpoints.LeaderEpochCheckpointFile
|
import kafka.server.checkpoints.LeaderEpochCheckpointFile
|
||||||
import kafka.server.epoch.LeaderEpochFileCache
|
import kafka.server.epoch.LeaderEpochFileCache
|
||||||
|
import kafka.server.{BrokerTopicMetrics, BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, LogDirFailureChannel, LogOffsetMetadata, OffsetAndEpoch, PartitionMetadataFile, RequestLocal}
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.errors._
|
import org.apache.kafka.common.errors._
|
||||||
import org.apache.kafka.common.internals.Topic
|
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.ListOffsetsRequest
|
||||||
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET
|
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.RecordError
|
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.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid}
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
import org.apache.kafka.server.common.MetadataVersion
|
||||||
import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_0_IV0
|
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.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig
|
||||||
import org.apache.kafka.server.record.BrokerCompressionType
|
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 */
|
/* A lock that guards all modifications to the log */
|
||||||
private val lock = new Object
|
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
|
/* 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
|
* 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,
|
def appendAsLeader(records: MemoryRecords,
|
||||||
leaderEpoch: Int,
|
leaderEpoch: Int,
|
||||||
origin: AppendOrigin = AppendOrigin.Client,
|
origin: AppendOrigin = AppendOrigin.CLIENT,
|
||||||
interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest,
|
interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest,
|
||||||
requestLocal: RequestLocal = RequestLocal.NoCaching): LogAppendInfo = {
|
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)
|
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 = {
|
def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
|
||||||
append(records,
|
append(records,
|
||||||
origin = AppendOrigin.Replication,
|
origin = AppendOrigin.REPLICATION,
|
||||||
interBrokerProtocolVersion = MetadataVersion.latest,
|
interBrokerProtocolVersion = MetadataVersion.latest,
|
||||||
validateAndAssignOffsets = false,
|
validateAndAssignOffsets = false,
|
||||||
leaderEpoch = -1,
|
leaderEpoch = -1,
|
||||||
|
@ -829,15 +829,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
localLog.checkIfMemoryMappedBufferClosed()
|
localLog.checkIfMemoryMappedBufferClosed()
|
||||||
if (validateAndAssignOffsets) {
|
if (validateAndAssignOffsets) {
|
||||||
// assign offsets to the message set
|
// assign offsets to the message set
|
||||||
val offset = new LongRef(localLog.logEndOffset)
|
val offset = PrimitiveRef.ofLong(localLog.logEndOffset)
|
||||||
appendInfo.firstOffset = Some(LogOffsetMetadata(offset.value))
|
appendInfo.firstOffset = Some(LogOffsetMetadata(offset.value))
|
||||||
val now = time.milliseconds
|
|
||||||
val validateAndOffsetAssignResult = try {
|
val validateAndOffsetAssignResult = try {
|
||||||
LogValidator.validateMessagesAndAssignOffsets(validRecords,
|
val validator = new LogValidator(validRecords,
|
||||||
topicPartition,
|
topicPartition,
|
||||||
offset,
|
|
||||||
time,
|
time,
|
||||||
now,
|
|
||||||
appendInfo.sourceCompression,
|
appendInfo.sourceCompression,
|
||||||
appendInfo.targetCompression,
|
appendInfo.targetCompression,
|
||||||
config.compact,
|
config.compact,
|
||||||
|
@ -846,21 +843,26 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
config.messageTimestampDifferenceMaxMs,
|
config.messageTimestampDifferenceMaxMs,
|
||||||
leaderEpoch,
|
leaderEpoch,
|
||||||
origin,
|
origin,
|
||||||
interBrokerProtocolVersion,
|
interBrokerProtocolVersion
|
||||||
brokerTopicStats,
|
)
|
||||||
|
validator.validateMessagesAndAssignOffsets(offset,
|
||||||
|
validatorMetricsRecorder,
|
||||||
requestLocal.getOrElse(throw new IllegalArgumentException(
|
requestLocal.getOrElse(throw new IllegalArgumentException(
|
||||||
"requestLocal should be defined if assignOffsets is true")))
|
"requestLocal should be defined if assignOffsets is true")
|
||||||
|
).bufferSupplier
|
||||||
|
)
|
||||||
} catch {
|
} catch {
|
||||||
case e: IOException =>
|
case e: IOException =>
|
||||||
throw new KafkaException(s"Error validating messages while appending to log $name", e)
|
throw new KafkaException(s"Error validating messages while appending to log $name", e)
|
||||||
}
|
}
|
||||||
|
|
||||||
validRecords = validateAndOffsetAssignResult.validatedRecords
|
validRecords = validateAndOffsetAssignResult.validatedRecords
|
||||||
appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp
|
appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestampMs
|
||||||
appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp
|
appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestampMs
|
||||||
appendInfo.lastOffset = offset.value - 1
|
appendInfo.lastOffset = offset.value - 1
|
||||||
appendInfo.recordConversionStats = validateAndOffsetAssignResult.recordConversionStats
|
appendInfo.recordConversionStats = validateAndOffsetAssignResult.recordConversionStats
|
||||||
if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
|
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
|
// re-validate message sizes if there's a possibility that they have changed (due to re-compression or message
|
||||||
// format conversion)
|
// format conversion)
|
||||||
|
@ -1070,7 +1072,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
if (batch.hasProducerId) {
|
if (batch.hasProducerId) {
|
||||||
// if this is a client produce request, there will be up to 5 batches which could have been duplicated.
|
// 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 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)
|
val maybeLastEntry = producerStateManager.lastEntry(batch.producerId)
|
||||||
|
|
||||||
maybeLastEntry.flatMap(_.findDuplicateBatch(batch)).foreach { duplicate =>
|
maybeLastEntry.flatMap(_.findDuplicateBatch(batch)).foreach { duplicate =>
|
||||||
|
@ -1129,11 +1131,11 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
var lastOffsetOfFirstBatch = -1L
|
var lastOffsetOfFirstBatch = -1L
|
||||||
|
|
||||||
records.batches.forEach { batch =>
|
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")
|
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
|
// 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 " +
|
throw new InvalidRecordException(s"The baseOffset of the record batch in the append to $topicPartition should " +
|
||||||
s"be 0, but it is ${batch.baseOffset}")
|
s"be 0, but it is ${batch.baseOffset}")
|
||||||
|
|
||||||
|
@ -1955,7 +1957,7 @@ object UnifiedLog extends Logging {
|
||||||
batch,
|
batch,
|
||||||
loadedProducers,
|
loadedProducers,
|
||||||
firstOffsetMetadata = None,
|
firstOffsetMetadata = None,
|
||||||
origin = AppendOrigin.Replication)
|
origin = AppendOrigin.REPLICATION)
|
||||||
maybeCompletedTxn.foreach(completedTxns += _)
|
maybeCompletedTxn.foreach(completedTxns += _)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -2165,7 +2167,9 @@ object UnifiedLog extends Logging {
|
||||||
parentDir: String,
|
parentDir: String,
|
||||||
topicPartition: TopicPartition): Unit = {
|
topicPartition: TopicPartition): Unit = {
|
||||||
val snapshotsToDelete = segments.flatMap { segment =>
|
val snapshotsToDelete = segments.flatMap { segment =>
|
||||||
producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset)}
|
producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset)
|
||||||
|
}
|
||||||
|
|
||||||
def deleteProducerSnapshots(): Unit = {
|
def deleteProducerSnapshots(): Unit = {
|
||||||
LocalLog.maybeHandleIOException(logDirFailureChannel,
|
LocalLog.maybeHandleIOException(logDirFailureChannel,
|
||||||
parentDir,
|
parentDir,
|
||||||
|
@ -2185,6 +2189,27 @@ object UnifiedLog extends Logging {
|
||||||
private[log] def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = {
|
private[log] def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = {
|
||||||
LocalLog.createNewCleanedSegment(dir, logConfig, baseOffset)
|
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 {
|
object LogMetricNames {
|
||||||
|
|
|
@ -16,7 +16,7 @@
|
||||||
*/
|
*/
|
||||||
package kafka.raft
|
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.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp}
|
||||||
import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, KafkaConfig, LogDirFailureChannel, RequestLocal}
|
import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, KafkaConfig, LogDirFailureChannel, RequestLocal}
|
||||||
import kafka.utils.{CoreUtils, Logging, Scheduler}
|
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.utils.{BufferSupplier, Time}
|
||||||
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
|
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.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 org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
|
||||||
|
|
||||||
import java.io.File
|
import java.io.File
|
||||||
|
@ -79,7 +80,7 @@ final class KafkaMetadataLog private (
|
||||||
handleAndConvertLogAppendInfo(
|
handleAndConvertLogAppendInfo(
|
||||||
log.appendAsLeader(records.asInstanceOf[MemoryRecords],
|
log.appendAsLeader(records.asInstanceOf[MemoryRecords],
|
||||||
leaderEpoch = epoch,
|
leaderEpoch = epoch,
|
||||||
origin = AppendOrigin.RaftLeader,
|
origin = AppendOrigin.RAFT_LEADER,
|
||||||
requestLocal = RequestLocal.NoCaching
|
requestLocal = RequestLocal.NoCaching
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
|
@ -23,7 +23,6 @@ import kafka.common.OffsetAndMetadata
|
||||||
import kafka.controller.ReplicaAssignment
|
import kafka.controller.ReplicaAssignment
|
||||||
import kafka.coordinator.group._
|
import kafka.coordinator.group._
|
||||||
import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
|
import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
|
||||||
import kafka.log.AppendOrigin
|
|
||||||
import kafka.network.RequestChannel
|
import kafka.network.RequestChannel
|
||||||
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
|
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
|
||||||
import kafka.server.metadata.ConfigRepository
|
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.authorizer._
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
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.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 org.apache.kafka.server.record.BrokerCompressionType
|
||||||
|
|
||||||
import java.lang.{Long => JLong}
|
import java.lang.{Long => JLong}
|
||||||
|
@ -669,7 +669,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
timeout = produceRequest.timeout.toLong,
|
timeout = produceRequest.timeout.toLong,
|
||||||
requiredAcks = produceRequest.acks,
|
requiredAcks = produceRequest.acks,
|
||||||
internalTopicsAllowed = internalTopicsAllowed,
|
internalTopicsAllowed = internalTopicsAllowed,
|
||||||
origin = AppendOrigin.Client,
|
origin = AppendOrigin.CLIENT,
|
||||||
entriesPerPartition = authorizedRequestInfo,
|
entriesPerPartition = authorizedRequestInfo,
|
||||||
requestLocal = requestLocal,
|
requestLocal = requestLocal,
|
||||||
responseCallback = sendResponseCallback,
|
responseCallback = sendResponseCallback,
|
||||||
|
@ -2333,7 +2333,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
timeout = config.requestTimeoutMs.toLong,
|
timeout = config.requestTimeoutMs.toLong,
|
||||||
requiredAcks = -1,
|
requiredAcks = -1,
|
||||||
internalTopicsAllowed = true,
|
internalTopicsAllowed = true,
|
||||||
origin = AppendOrigin.Coordinator,
|
origin = AppendOrigin.COORDINATOR,
|
||||||
entriesPerPartition = controlRecords,
|
entriesPerPartition = controlRecords,
|
||||||
requestLocal = requestLocal,
|
requestLocal = requestLocal,
|
||||||
responseCallback = maybeSendResponseCallback(producerId, marker.transactionResult))
|
responseCallback = maybeSendResponseCallback(producerId, marker.transactionResult))
|
||||||
|
|
|
@ -24,7 +24,6 @@ import java.util.concurrent.locks.Lock
|
||||||
import com.yammer.metrics.core.Meter
|
import com.yammer.metrics.core.Meter
|
||||||
import kafka.api._
|
import kafka.api._
|
||||||
import kafka.cluster.{BrokerEndPoint, Partition}
|
import kafka.cluster.{BrokerEndPoint, Partition}
|
||||||
import kafka.common.RecordValidationException
|
|
||||||
import kafka.controller.{KafkaController, StateChangeLogger}
|
import kafka.controller.{KafkaController, StateChangeLogger}
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.log.remote.RemoteLogManager
|
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.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
|
||||||
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
|
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
|
||||||
import org.apache.kafka.server.common.MetadataVersion._
|
import org.apache.kafka.server.common.MetadataVersion._
|
||||||
|
import org.apache.kafka.server.log.internals.{AppendOrigin, RecordValidationException}
|
||||||
|
|
||||||
import java.nio.file.{Files, Paths}
|
import java.nio.file.{Files, Paths}
|
||||||
import java.util
|
import java.util
|
||||||
|
@ -984,7 +984,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
val logStartOffset = processFailedRecord(topicPartition, rve.invalidException)
|
val logStartOffset = processFailedRecord(topicPartition, rve.invalidException)
|
||||||
val recordErrors = rve.recordErrors
|
val recordErrors = rve.recordErrors
|
||||||
(topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(
|
(topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(
|
||||||
logStartOffset, recordErrors, rve.invalidException.getMessage), Some(rve.invalidException)))
|
logStartOffset, recordErrors.asScala, rve.invalidException.getMessage), Some(rve.invalidException)))
|
||||||
case t: Throwable =>
|
case t: Throwable =>
|
||||||
val logStartOffset = processFailedRecord(topicPartition, t)
|
val logStartOffset = processFailedRecord(topicPartition, t)
|
||||||
(topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t)))
|
(topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t)))
|
||||||
|
|
|
@ -18,7 +18,6 @@
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import kafka.cluster.BrokerEndPoint
|
import kafka.cluster.BrokerEndPoint
|
||||||
import kafka.log.AppendOrigin
|
|
||||||
import kafka.server.checkpoints.LazyOffsetCheckpoints
|
import kafka.server.checkpoints.LazyOffsetCheckpoints
|
||||||
import kafka.utils.{MockScheduler, MockTime, TestUtils}
|
import kafka.utils.{MockScheduler, MockTime, TestUtils}
|
||||||
import org.apache.kafka.common.{Node, TopicPartition, Uuid}
|
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.record.{CompressionType, MemoryRecords, SimpleRecord}
|
||||||
import org.apache.kafka.common.requests.LeaderAndIsrRequest
|
import org.apache.kafka.common.requests.LeaderAndIsrRequest
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
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.{BeforeEach, Test}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.mockito.Mockito.mock
|
import org.mockito.Mockito.mock
|
||||||
|
@ -158,7 +158,7 @@ class LocalLeaderEndPointTest {
|
||||||
private def appendRecords(replicaManager: ReplicaManager,
|
private def appendRecords(replicaManager: ReplicaManager,
|
||||||
partition: TopicPartition,
|
partition: TopicPartition,
|
||||||
records: MemoryRecords,
|
records: MemoryRecords,
|
||||||
origin: AppendOrigin = AppendOrigin.Client,
|
origin: AppendOrigin = AppendOrigin.CLIENT,
|
||||||
requiredAcks: Short = -1): CallbackResult[PartitionResponse] = {
|
requiredAcks: Short = -1): CallbackResult[PartitionResponse] = {
|
||||||
val result = new CallbackResult[PartitionResponse]()
|
val result = new CallbackResult[PartitionResponse]()
|
||||||
def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = {
|
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.utils.Utils
|
||||||
import org.apache.kafka.common.{TopicPartition, Uuid}
|
import org.apache.kafka.common.{TopicPartition, Uuid}
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
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.Assertions.{assertEquals, assertFalse, assertTrue}
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||||
import org.mockito.ArgumentMatchers
|
import org.mockito.ArgumentMatchers
|
||||||
|
@ -369,7 +370,7 @@ class PartitionLockTest extends Logging {
|
||||||
(0 until numRecords).foreach { _ =>
|
(0 until numRecords).foreach { _ =>
|
||||||
val batch = TestUtils.records(records = List(new SimpleRecord("k1".getBytes, "v1".getBytes),
|
val batch = TestUtils.records(records = List(new SimpleRecord("k1".getBytes, "v1".getBytes),
|
||||||
new SimpleRecord("k2".getBytes, "v2".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.common.security.auth.{KafkaPrincipal, SecurityProtocol}
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
import org.apache.kafka.server.common.MetadataVersion
|
||||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
|
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.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
import org.junit.jupiter.params.provider.ValueSource
|
||||||
|
@ -740,8 +741,8 @@ class PartitionTest extends AbstractPartitionTest {
|
||||||
val requestLocal = RequestLocal.withThreadConfinedCaching
|
val requestLocal = RequestLocal.withThreadConfinedCaching
|
||||||
// after makeLeader(() call, partition should know about all the replicas
|
// after makeLeader(() call, partition should know about all the replicas
|
||||||
// append records with initial leader epoch
|
// append records with initial leader epoch
|
||||||
partition.appendRecordsToLeader(batch1, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal)
|
partition.appendRecordsToLeader(batch1, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal)
|
||||||
partition.appendRecordsToLeader(batch2, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal)
|
partition.appendRecordsToLeader(batch2, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal)
|
||||||
assertEquals(partition.localLogOrException.logStartOffset, partition.localLogOrException.highWatermark,
|
assertEquals(partition.localLogOrException.logStartOffset, partition.localLogOrException.highWatermark,
|
||||||
"Expected leader's HW not move")
|
"Expected leader's HW not move")
|
||||||
|
|
||||||
|
@ -949,7 +950,7 @@ class PartitionTest extends AbstractPartitionTest {
|
||||||
new SimpleRecord("k2".getBytes, "v2".getBytes),
|
new SimpleRecord("k2".getBytes, "v2".getBytes),
|
||||||
new SimpleRecord("k3".getBytes, "v3".getBytes)),
|
new SimpleRecord("k3".getBytes, "v3".getBytes)),
|
||||||
baseOffset = 0L)
|
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 = {
|
def fetchOffset(isolationLevel: Option[IsolationLevel], timestamp: Long): TimestampAndOffset = {
|
||||||
val res = partition.fetchOffsetForTimestamp(timestamp,
|
val res = partition.fetchOffsetForTimestamp(timestamp,
|
||||||
|
@ -1075,9 +1076,9 @@ class PartitionTest extends AbstractPartitionTest {
|
||||||
|
|
||||||
// after makeLeader(() call, partition should know about all the replicas
|
// after makeLeader(() call, partition should know about all the replicas
|
||||||
// append records with initial leader epoch
|
// 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
|
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")
|
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
|
// 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
|
val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset
|
||||||
|
|
||||||
// append records with the latest leader epoch
|
// 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
|
// fetch from follower not in ISR from log start offset should not add this follower to ISR
|
||||||
fetchFollower(partition, replicaId = follower1, fetchOffset = 0)
|
fetchFollower(partition, replicaId = follower1, fetchOffset = 0)
|
||||||
|
|
|
@ -21,9 +21,8 @@ import java.util.concurrent.{ConcurrentHashMap, Executors}
|
||||||
import java.util.{Collections, Random}
|
import java.util.{Collections, Random}
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
import java.util.concurrent.locks.Lock
|
import java.util.concurrent.locks.Lock
|
||||||
|
|
||||||
import kafka.coordinator.AbstractCoordinatorConcurrencyTest._
|
import kafka.coordinator.AbstractCoordinatorConcurrencyTest._
|
||||||
import kafka.log.{AppendOrigin, LogConfig, UnifiedLog}
|
import kafka.log.{LogConfig, UnifiedLog}
|
||||||
import kafka.server._
|
import kafka.server._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import kafka.utils.timer.MockTimer
|
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.protocol.Errors
|
||||||
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordConversionStats}
|
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordConversionStats}
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
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.junit.jupiter.api.{AfterEach, BeforeEach}
|
||||||
import org.mockito.Mockito.{mock, withSettings, CALLS_REAL_METHODS}
|
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.TimeUnit
|
||||||
import java.util.concurrent.locks.ReentrantLock
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
import kafka.cluster.Partition
|
import kafka.cluster.Partition
|
||||||
import kafka.log.AppendOrigin
|
|
||||||
import kafka.zk.KafkaZkClient
|
import kafka.zk.KafkaZkClient
|
||||||
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription
|
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
||||||
import org.apache.kafka.common.internals.Topic
|
import org.apache.kafka.common.internals.Topic
|
||||||
import org.apache.kafka.common.metrics.Metrics
|
import org.apache.kafka.common.metrics.Metrics
|
||||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
|
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.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
|
@ -3854,7 +3854,7 @@ class GroupCoordinatorTest {
|
||||||
when(replicaManager.appendRecords(anyLong,
|
when(replicaManager.appendRecords(anyLong,
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
capturedArgument.capture(),
|
capturedArgument.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -3888,7 +3888,7 @@ class GroupCoordinatorTest {
|
||||||
when(replicaManager.appendRecords(anyLong,
|
when(replicaManager.appendRecords(anyLong,
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
capturedArgument.capture(),
|
capturedArgument.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -4032,7 +4032,7 @@ class GroupCoordinatorTest {
|
||||||
when(replicaManager.appendRecords(anyLong,
|
when(replicaManager.appendRecords(anyLong,
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
capturedArgument.capture(),
|
capturedArgument.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -4065,7 +4065,7 @@ class GroupCoordinatorTest {
|
||||||
when(replicaManager.appendRecords(anyLong,
|
when(replicaManager.appendRecords(anyLong,
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
capturedArgument.capture(),
|
capturedArgument.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
|
|
@ -21,12 +21,12 @@ import java.lang.management.ManagementFactory
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.util.concurrent.locks.ReentrantLock
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
import java.util.{Collections, Optional}
|
import java.util.{Collections, Optional}
|
||||||
|
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
|
|
||||||
import javax.management.ObjectName
|
import javax.management.ObjectName
|
||||||
import kafka.cluster.Partition
|
import kafka.cluster.Partition
|
||||||
import kafka.common.OffsetAndMetadata
|
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.server.{FetchDataInfo, FetchLogEnd, HostedPartition, KafkaConfig, LogOffsetMetadata, ReplicaManager, RequestLocal}
|
||||||
import kafka.utils.{KafkaScheduler, MockTime, TestUtils}
|
import kafka.utils.{KafkaScheduler, MockTime, TestUtils}
|
||||||
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
|
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.common.utils.Utils
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
import org.apache.kafka.server.common.MetadataVersion
|
||||||
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.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||||
|
@ -1172,7 +1173,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1208,7 +1209,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1282,7 +1283,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1322,7 +1323,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
capturedResponseCallback.capture(),
|
capturedResponseCallback.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1380,7 +1381,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1428,7 +1429,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1561,7 +1562,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1640,7 +1641,7 @@ class GroupMetadataManagerTest {
|
||||||
time.sleep(2)
|
time.sleep(2)
|
||||||
|
|
||||||
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -1655,7 +1656,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1681,7 +1682,7 @@ class GroupMetadataManagerTest {
|
||||||
when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE))
|
when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE))
|
||||||
mockGetPartition()
|
mockGetPartition()
|
||||||
when(partition.appendRecordsToLeader(recordsCapture.capture(),
|
when(partition.appendRecordsToLeader(recordsCapture.capture(),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -1724,7 +1725,7 @@ class GroupMetadataManagerTest {
|
||||||
when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE))
|
when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE))
|
||||||
mockGetPartition()
|
mockGetPartition()
|
||||||
when(partition.appendRecordsToLeader(recordsCapture.capture(),
|
when(partition.appendRecordsToLeader(recordsCapture.capture(),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -1792,7 +1793,7 @@ class GroupMetadataManagerTest {
|
||||||
val recordsCapture: ArgumentCaptor[MemoryRecords] = ArgumentCaptor.forClass(classOf[MemoryRecords])
|
val recordsCapture: ArgumentCaptor[MemoryRecords] = ArgumentCaptor.forClass(classOf[MemoryRecords])
|
||||||
|
|
||||||
when(partition.appendRecordsToLeader(recordsCapture.capture(),
|
when(partition.appendRecordsToLeader(recordsCapture.capture(),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -1886,7 +1887,7 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the offset tombstone
|
// expect the offset tombstone
|
||||||
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -1907,7 +1908,7 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the offset tombstone
|
// expect the offset tombstone
|
||||||
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -1947,7 +1948,7 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the offset tombstone
|
// expect the offset tombstone
|
||||||
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -2020,7 +2021,7 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the offset tombstone
|
// expect the offset tombstone
|
||||||
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
@ -2144,13 +2145,13 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the offset tombstone
|
// expect the offset tombstone
|
||||||
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
when(partition.appendRecordsToLeader(any[MemoryRecords],
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
|
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
verify(partition).appendRecordsToLeader(any[MemoryRecords],
|
verify(partition).appendRecordsToLeader(any[MemoryRecords],
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(),
|
||||||
any())
|
any())
|
||||||
verify(replicaManager, times(2)).onlinePartition(groupTopicPartition)
|
verify(replicaManager, times(2)).onlinePartition(groupTopicPartition)
|
||||||
|
|
||||||
|
@ -2454,7 +2455,7 @@ class GroupMetadataManagerTest {
|
||||||
verify(replicaManager).appendRecords(anyLong(),
|
verify(replicaManager).appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
capturedArgument.capture(),
|
capturedArgument.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -2469,7 +2470,7 @@ class GroupMetadataManagerTest {
|
||||||
when(replicaManager.appendRecords(anyLong(),
|
when(replicaManager.appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
capturedRecords.capture(),
|
capturedRecords.capture(),
|
||||||
capturedCallback.capture(),
|
capturedCallback.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
|
|
@ -20,9 +20,8 @@ import java.lang.management.ManagementFactory
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.util.concurrent.CountDownLatch
|
import java.util.concurrent.CountDownLatch
|
||||||
import java.util.concurrent.locks.ReentrantLock
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
|
|
||||||
import javax.management.ObjectName
|
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.server.{FetchDataInfo, FetchLogEnd, LogOffsetMetadata, ReplicaManager, RequestLocal}
|
||||||
import kafka.utils.{MockScheduler, Pool, TestUtils}
|
import kafka.utils.{MockScheduler, Pool, TestUtils}
|
||||||
import kafka.zk.KafkaZkClient
|
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.ProduceResponse.PartitionResponse
|
||||||
import org.apache.kafka.common.requests.TransactionResult
|
import org.apache.kafka.common.requests.TransactionResult
|
||||||
import org.apache.kafka.common.utils.MockTime
|
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.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||||
import org.mockito.{ArgumentCaptor, ArgumentMatchers}
|
import org.mockito.{ArgumentCaptor, ArgumentMatchers}
|
||||||
|
@ -648,7 +648,7 @@ class TransactionStateManagerTest {
|
||||||
anyLong(),
|
anyLong(),
|
||||||
ArgumentMatchers.eq((-1).toShort),
|
ArgumentMatchers.eq((-1).toShort),
|
||||||
ArgumentMatchers.eq(true),
|
ArgumentMatchers.eq(true),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -691,7 +691,7 @@ class TransactionStateManagerTest {
|
||||||
anyLong(),
|
anyLong(),
|
||||||
ArgumentMatchers.eq((-1).toShort),
|
ArgumentMatchers.eq((-1).toShort),
|
||||||
ArgumentMatchers.eq(true),
|
ArgumentMatchers.eq(true),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -731,7 +731,7 @@ class TransactionStateManagerTest {
|
||||||
anyLong(),
|
anyLong(),
|
||||||
ArgumentMatchers.eq((-1).toShort),
|
ArgumentMatchers.eq((-1).toShort),
|
||||||
ArgumentMatchers.eq(true),
|
ArgumentMatchers.eq(true),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -886,7 +886,7 @@ class TransactionStateManagerTest {
|
||||||
anyLong(),
|
anyLong(),
|
||||||
ArgumentMatchers.eq((-1).toShort),
|
ArgumentMatchers.eq((-1).toShort),
|
||||||
ArgumentMatchers.eq(true),
|
ArgumentMatchers.eq(true),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
recordsCapture.capture(),
|
recordsCapture.capture(),
|
||||||
callbackCapture.capture(),
|
callbackCapture.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
@ -1036,7 +1036,7 @@ class TransactionStateManagerTest {
|
||||||
when(replicaManager.appendRecords(anyLong(),
|
when(replicaManager.appendRecords(anyLong(),
|
||||||
anyShort(),
|
anyShort(),
|
||||||
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
internalTopicsAllowed = ArgumentMatchers.eq(true),
|
||||||
origin = ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any[Map[TopicPartition, MemoryRecords]],
|
any[Map[TopicPartition, MemoryRecords]],
|
||||||
capturedArgument.capture(),
|
capturedArgument.capture(),
|
||||||
any[Option[ReentrantLock]],
|
any[Option[ReentrantLock]],
|
||||||
|
|
|
@ -20,12 +20,12 @@ package kafka.log
|
||||||
import java.io.File
|
import java.io.File
|
||||||
import java.nio.file.Files
|
import java.nio.file.Files
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
|
||||||
import kafka.server.{BrokerTopicStats, LogDirFailureChannel}
|
import kafka.server.{BrokerTopicStats, LogDirFailureChannel}
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.record._
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.Utils
|
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.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, Test}
|
import org.junit.jupiter.api.{AfterEach, Test}
|
||||||
|
|
||||||
|
@ -678,7 +678,7 @@ class LogCleanerManagerTest extends Logging {
|
||||||
|
|
||||||
log.appendAsLeader(MemoryRecords.withEndTransactionMarker(time.milliseconds(), producerId, producerEpoch,
|
log.appendAsLeader(MemoryRecords.withEndTransactionMarker(time.milliseconds(), producerId, producerEpoch,
|
||||||
new EndTransactionMarker(ControlRecordType.ABORT, 15)), leaderEpoch = 0,
|
new EndTransactionMarker(ControlRecordType.ABORT, 15)), leaderEpoch = 0,
|
||||||
origin = AppendOrigin.Coordinator)
|
origin = AppendOrigin.COORDINATOR)
|
||||||
log.roll()
|
log.roll()
|
||||||
log.updateHighWatermark(4L)
|
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.errors.CorruptRecordException
|
||||||
import org.apache.kafka.common.record._
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.Utils
|
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.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, Test}
|
import org.junit.jupiter.api.{AfterEach, Test}
|
||||||
|
|
||||||
|
@ -303,11 +303,11 @@ class LogCleanerTest {
|
||||||
val appendProducer2 = appendTransactionalAsLeader(log, producerId2, producerEpoch)
|
val appendProducer2 = appendTransactionalAsLeader(log, producerId2, producerEpoch)
|
||||||
|
|
||||||
def abort(producerId: Long): Unit = {
|
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 = {
|
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)
|
// Append some transaction data (offset range in parenthesis)
|
||||||
|
@ -393,10 +393,10 @@ class LogCleanerTest {
|
||||||
appendProducer1(Seq(1, 2))
|
appendProducer1(Seq(1, 2))
|
||||||
appendProducer2(Seq(2, 3))
|
appendProducer2(Seq(2, 3))
|
||||||
appendProducer1(Seq(3, 4))
|
appendProducer1(Seq(3, 4))
|
||||||
log.appendAsLeader(abortMarker(pid1, 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)
|
log.appendAsLeader(commitMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR)
|
||||||
appendProducer1(Seq(2))
|
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)
|
val abortedTransactions = log.collectAbortedTransactions(log.logStartOffset, log.logEndOffset)
|
||||||
|
|
||||||
|
@ -434,11 +434,11 @@ class LogCleanerTest {
|
||||||
appendProducer2(Seq(5, 6))
|
appendProducer2(Seq(5, 6))
|
||||||
appendProducer3(Seq(6, 7))
|
appendProducer3(Seq(6, 7))
|
||||||
appendProducer1(Seq(7, 8))
|
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))
|
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))
|
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
|
// 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
|
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(1))
|
||||||
appendProducer(Seq(2, 3))
|
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))
|
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()
|
log.roll()
|
||||||
|
|
||||||
// cannot remove the marker in this pass because there are still valid records
|
// 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))
|
assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log))
|
||||||
|
|
||||||
appendProducer(Seq(1, 3))
|
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()
|
log.roll()
|
||||||
|
|
||||||
// the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction
|
// 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)
|
val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch)
|
||||||
|
|
||||||
appendProducer(Seq(1))
|
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))
|
||||||
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()
|
log.roll()
|
||||||
|
|
||||||
cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp)
|
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}]
|
// [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}]
|
||||||
producer2(Seq(2, 3)) // offsets 2, 3
|
producer2(Seq(2, 3)) // offsets 2, 3
|
||||||
log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0,
|
log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0,
|
||||||
origin = AppendOrigin.Coordinator) // offset 4
|
origin = AppendOrigin.COORDINATOR) // offset 4
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
// [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}], [{2}, {3}, {Producer1: Commit}]
|
// [{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(2, 2), leaderEpoch = 0) // offset 5
|
||||||
log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 6
|
log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 6
|
||||||
log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0,
|
log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0,
|
||||||
origin = AppendOrigin.Coordinator) // offset 7
|
origin = AppendOrigin.COORDINATOR) // offset 7
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
// first time through the records are removed
|
// first time through the records are removed
|
||||||
|
@ -582,7 +582,7 @@ class LogCleanerTest {
|
||||||
// {1}, {3}, {4}, {5}, {6}, {7}, {8}, {9} ==> Offsets
|
// {1}, {3}, {4}, {5}, {6}, {7}, {8}, {9} ==> Offsets
|
||||||
producer2(Seq(1)) // offset 8
|
producer2(Seq(1)) // offset 8
|
||||||
log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0,
|
log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0,
|
||||||
origin = AppendOrigin.Coordinator) // offset 9
|
origin = AppendOrigin.COORDINATOR) // offset 9
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
// Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}]
|
// Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}]
|
||||||
|
@ -612,7 +612,7 @@ class LogCleanerTest {
|
||||||
|
|
||||||
// [{Producer1: Commit}, {2}, {3}]
|
// [{Producer1: Commit}, {2}, {3}]
|
||||||
log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0,
|
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(2, 2), leaderEpoch = 0) // offset 2
|
||||||
log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 3
|
log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 3
|
||||||
log.roll()
|
log.roll()
|
||||||
|
@ -648,7 +648,7 @@ class LogCleanerTest {
|
||||||
appendTransaction(Seq(1))
|
appendTransaction(Seq(1))
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator)
|
log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR)
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
// Both the record and the marker should remain after cleaning
|
// Both the record and the marker should remain after cleaning
|
||||||
|
@ -671,7 +671,7 @@ class LogCleanerTest {
|
||||||
appendTransaction(Seq(1))
|
appendTransaction(Seq(1))
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator)
|
log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR)
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
// Both the batch and the marker should remain after cleaning. The batch is retained
|
// 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(1))
|
||||||
appendProducer(Seq(2, 3))
|
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))
|
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()
|
log.roll()
|
||||||
|
|
||||||
// Aborted records are removed, but the abort marker is still preserved.
|
// 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 log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch,
|
val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch,
|
||||||
origin = AppendOrigin.Replication)
|
origin = AppendOrigin.REPLICATION)
|
||||||
appendFirstTransaction(Seq(1))
|
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,
|
val appendSecondTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch,
|
||||||
origin = AppendOrigin.Replication)
|
origin = AppendOrigin.REPLICATION)
|
||||||
appendSecondTransaction(Seq(2))
|
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(1, 1), leaderEpoch = 0)
|
||||||
log.appendAsLeader(record(2, 1), leaderEpoch = 0)
|
log.appendAsLeader(record(2, 1), leaderEpoch = 0)
|
||||||
|
@ -771,7 +771,7 @@ class LogCleanerTest {
|
||||||
val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch)
|
val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch)
|
||||||
|
|
||||||
appendProducer(Seq(2, 3)) // batch last offset is 1
|
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()
|
log.roll()
|
||||||
|
|
||||||
def assertAbortedTransactionIndexed(): Unit = {
|
def assertAbortedTransactionIndexed(): Unit = {
|
||||||
|
@ -1012,7 +1012,7 @@ class LogCleanerTest {
|
||||||
|
|
||||||
appendProducer(Seq(1))
|
appendProducer(Seq(1))
|
||||||
appendProducer(Seq(2, 3))
|
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()
|
log.roll()
|
||||||
|
|
||||||
cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset))
|
cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset))
|
||||||
|
@ -1046,7 +1046,7 @@ class LogCleanerTest {
|
||||||
val producerId1 = 1L
|
val producerId1 = 1L
|
||||||
val appendProducer = appendTransactionalAsLeader(log, producerId1, producerEpoch, leaderEpoch)
|
val appendProducer = appendTransactionalAsLeader(log, producerId1, producerEpoch, leaderEpoch)
|
||||||
appendProducer(Seq(1))
|
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
|
// Now we append one transaction with a key which conflicts with the COMMIT marker appended above
|
||||||
def commitRecordKey(): ByteBuffer = {
|
def commitRecordKey(): ByteBuffer = {
|
||||||
|
@ -1065,8 +1065,8 @@ class LogCleanerTest {
|
||||||
0,
|
0,
|
||||||
new SimpleRecord(time.milliseconds(), commitRecordKey(), ByteBuffer.wrap("foo".getBytes))
|
new SimpleRecord(time.milliseconds(), commitRecordKey(), ByteBuffer.wrap("foo".getBytes))
|
||||||
)
|
)
|
||||||
log.appendAsLeader(records, leaderEpoch, origin = AppendOrigin.Client)
|
log.appendAsLeader(records, leaderEpoch, origin = AppendOrigin.CLIENT)
|
||||||
log.appendAsLeader(commitMarker(producerId2, producerEpoch), leaderEpoch, origin = AppendOrigin.Coordinator)
|
log.appendAsLeader(commitMarker(producerId2, producerEpoch), leaderEpoch, origin = AppendOrigin.COORDINATOR)
|
||||||
log.roll()
|
log.roll()
|
||||||
assertEquals(List(0, 1, 2, 3), offsetsInLog(log))
|
assertEquals(List(0, 1, 2, 3), offsetsInLog(log))
|
||||||
|
|
||||||
|
@ -2013,7 +2013,7 @@ class LogCleanerTest {
|
||||||
producerId: Long,
|
producerId: Long,
|
||||||
producerEpoch: Short,
|
producerEpoch: Short,
|
||||||
leaderEpoch: Int = 0,
|
leaderEpoch: Int = 0,
|
||||||
origin: AppendOrigin = AppendOrigin.Client
|
origin: AppendOrigin = AppendOrigin.CLIENT
|
||||||
): Seq[Int] => LogAppendInfo = {
|
): Seq[Int] => LogAppendInfo = {
|
||||||
appendIdempotentAsLeader(
|
appendIdempotentAsLeader(
|
||||||
log,
|
log,
|
||||||
|
@ -2031,7 +2031,7 @@ class LogCleanerTest {
|
||||||
producerEpoch: Short,
|
producerEpoch: Short,
|
||||||
isTransactional: Boolean = false,
|
isTransactional: Boolean = false,
|
||||||
leaderEpoch: Int = 0,
|
leaderEpoch: Int = 0,
|
||||||
origin: AppendOrigin = AppendOrigin.Client
|
origin: AppendOrigin = AppendOrigin.CLIENT
|
||||||
): Seq[Int] => LogAppendInfo = {
|
): Seq[Int] => LogAppendInfo = {
|
||||||
var sequence = 0
|
var sequence = 0
|
||||||
keys: Seq[Int] => {
|
keys: Seq[Int] => {
|
||||||
|
|
|
@ -32,7 +32,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse}
|
||||||
import java.nio.file.Files
|
import java.nio.file.Files
|
||||||
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
|
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
|
||||||
import kafka.log
|
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.collection.Iterable
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -215,7 +215,7 @@ object LogTestUtils {
|
||||||
leaderEpoch: Int = 0): LogAppendInfo = {
|
leaderEpoch: Int = 0): LogAppendInfo = {
|
||||||
val records = endTxnRecords(controlType, producerId, producerEpoch,
|
val records = endTxnRecords(controlType, producerId, producerEpoch,
|
||||||
coordinatorEpoch = coordinatorEpoch, timestamp = timestamp)
|
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,
|
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.internals.Topic
|
||||||
import org.apache.kafka.common.record._
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.{MockTime, Utils}
|
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.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||||
import org.mockito.Mockito.{mock, when}
|
import org.mockito.Mockito.{mock, when}
|
||||||
|
@ -110,7 +110,7 @@ class ProducerStateManagerTest {
|
||||||
val epoch = 15.toShort
|
val epoch = 15.toShort
|
||||||
val sequence = Int.MaxValue
|
val sequence = Int.MaxValue
|
||||||
val offset = 735L
|
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)
|
append(stateManager, producerId, epoch, 0, offset + 500)
|
||||||
|
|
||||||
|
@ -128,7 +128,7 @@ class ProducerStateManagerTest {
|
||||||
def testProducerSequenceWithWrapAroundBatchRecord(): Unit = {
|
def testProducerSequenceWithWrapAroundBatchRecord(): Unit = {
|
||||||
val epoch = 15.toShort
|
val epoch = 15.toShort
|
||||||
|
|
||||||
val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Replication)
|
val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.REPLICATION)
|
||||||
// Sequence number wrap around
|
// Sequence number wrap around
|
||||||
appendInfo.appendDataBatch(epoch, Int.MaxValue - 10, 9, time.milliseconds(),
|
appendInfo.appendDataBatch(epoch, Int.MaxValue - 10, 9, time.milliseconds(),
|
||||||
LogOffsetMetadata(2000L), 2020L, isTransactional = false)
|
LogOffsetMetadata(2000L), 2020L, isTransactional = false)
|
||||||
|
@ -148,7 +148,7 @@ class ProducerStateManagerTest {
|
||||||
val epoch = 15.toShort
|
val epoch = 15.toShort
|
||||||
val sequence = Int.MaxValue
|
val sequence = Int.MaxValue
|
||||||
val offset = 735L
|
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))
|
assertThrows(classOf[OutOfOrderSequenceException], () => append(stateManager, producerId, epoch, 1, offset + 500))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -157,7 +157,7 @@ class ProducerStateManagerTest {
|
||||||
val epoch = 5.toShort
|
val epoch = 5.toShort
|
||||||
val sequence = 16
|
val sequence = 16
|
||||||
val offset = 735L
|
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)
|
val maybeLastEntry = stateManager.lastEntry(producerId)
|
||||||
assertTrue(maybeLastEntry.isDefined)
|
assertTrue(maybeLastEntry.isDefined)
|
||||||
|
@ -197,7 +197,7 @@ class ProducerStateManagerTest {
|
||||||
val producerEpoch = 0.toShort
|
val producerEpoch = 0.toShort
|
||||||
val offset = 992342L
|
val offset = 992342L
|
||||||
val seq = 0
|
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,
|
val firstOffsetMetadata = LogOffsetMetadata(messageOffset = offset, segmentBaseOffset = 990000L,
|
||||||
relativePositionInSegment = 234224)
|
relativePositionInSegment = 234224)
|
||||||
|
@ -235,7 +235,7 @@ class ProducerStateManagerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Start one transaction in a separate append
|
// 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)
|
appendData(16L, 20L, firstAppend)
|
||||||
assertEquals(new TxnMetadata(producerId, 16L), firstAppend.startedTransactions.head)
|
assertEquals(new TxnMetadata(producerId, 16L), firstAppend.startedTransactions.head)
|
||||||
stateManager.update(firstAppend)
|
stateManager.update(firstAppend)
|
||||||
|
@ -245,7 +245,7 @@ class ProducerStateManagerTest {
|
||||||
// Now do a single append which completes the old transaction, mixes in
|
// Now do a single append which completes the old transaction, mixes in
|
||||||
// some empty transactions, one non-empty complete transaction, and one
|
// some empty transactions, one non-empty complete transaction, and one
|
||||||
// incomplete transaction
|
// 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)
|
val firstCompletedTxn = appendEndTxn(ControlRecordType.COMMIT, 21, secondAppend)
|
||||||
assertEquals(Some(new CompletedTxn(producerId, 16L, 21, false)), firstCompletedTxn)
|
assertEquals(Some(new CompletedTxn(producerId, 16L, 21, false)), firstCompletedTxn)
|
||||||
assertEquals(None, appendEndTxn(ControlRecordType.COMMIT, 22, secondAppend))
|
assertEquals(None, appendEndTxn(ControlRecordType.COMMIT, 22, secondAppend))
|
||||||
|
@ -371,7 +371,7 @@ class ProducerStateManagerTest {
|
||||||
partition,
|
partition,
|
||||||
producerId,
|
producerId,
|
||||||
ProducerStateEntry.empty(producerId),
|
ProducerStateEntry.empty(producerId),
|
||||||
AppendOrigin.Client
|
AppendOrigin.CLIENT
|
||||||
)
|
)
|
||||||
val firstOffsetMetadata = LogOffsetMetadata(messageOffset = startOffset, segmentBaseOffset = segmentBaseOffset,
|
val firstOffsetMetadata = LogOffsetMetadata(messageOffset = startOffset, segmentBaseOffset = segmentBaseOffset,
|
||||||
relativePositionInSegment = 50 * relativeOffset)
|
relativePositionInSegment = 50 * relativeOffset)
|
||||||
|
@ -418,14 +418,14 @@ class ProducerStateManagerTest {
|
||||||
def testPrepareUpdateDoesNotMutate(): Unit = {
|
def testPrepareUpdateDoesNotMutate(): Unit = {
|
||||||
val producerEpoch = 0.toShort
|
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(),
|
appendInfo.appendDataBatch(producerEpoch, 0, 5, time.milliseconds(),
|
||||||
LogOffsetMetadata(15L), 20L, isTransactional = false)
|
LogOffsetMetadata(15L), 20L, isTransactional = false)
|
||||||
assertEquals(None, stateManager.lastEntry(producerId))
|
assertEquals(None, stateManager.lastEntry(producerId))
|
||||||
stateManager.update(appendInfo)
|
stateManager.update(appendInfo)
|
||||||
assertTrue(stateManager.lastEntry(producerId).isDefined)
|
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(),
|
nextAppendInfo.appendDataBatch(producerEpoch, 6, 10, time.milliseconds(),
|
||||||
LogOffsetMetadata(26L), 30L, isTransactional = false)
|
LogOffsetMetadata(26L), 30L, isTransactional = false)
|
||||||
assertTrue(stateManager.lastEntry(producerId).isDefined)
|
assertTrue(stateManager.lastEntry(producerId).isDefined)
|
||||||
|
@ -449,7 +449,7 @@ class ProducerStateManagerTest {
|
||||||
val offset = 9L
|
val offset = 9L
|
||||||
append(stateManager, producerId, producerEpoch, 0, offset)
|
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(),
|
appendInfo.appendDataBatch(producerEpoch, 1, 5, time.milliseconds(),
|
||||||
LogOffsetMetadata(16L), 20L, isTransactional = true)
|
LogOffsetMetadata(16L), 20L, isTransactional = true)
|
||||||
var lastEntry = appendInfo.toEntry
|
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
|
// 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.
|
// the append on a replica should be accepted with the local producer state updated to the appended value.
|
||||||
assertFalse(recoveredMapping.activeProducers.contains(producerId))
|
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))
|
assertTrue(recoveredMapping.activeProducers.contains(producerId))
|
||||||
val producerStateEntry = recoveredMapping.activeProducers.get(producerId).head
|
val producerStateEntry = recoveredMapping.activeProducers.get(producerId).head
|
||||||
assertEquals(epoch, producerStateEntry.producerEpoch)
|
assertEquals(epoch, producerStateEntry.producerEpoch)
|
||||||
|
@ -685,10 +685,10 @@ class ProducerStateManagerTest {
|
||||||
val outOfOrderSequence = 3
|
val outOfOrderSequence = 3
|
||||||
|
|
||||||
// First we ensure that we raise an OutOfOrderSequenceException is raised when the append comes from a client.
|
// 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)
|
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)
|
assertEquals(outOfOrderSequence, stateManager.activeProducers(producerId).lastSeq)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -905,9 +905,9 @@ class ProducerStateManagerTest {
|
||||||
|
|
||||||
val epoch = 0.toShort
|
val epoch = 0.toShort
|
||||||
append(stateManager, producerId, epoch, RecordBatch.NO_SEQUENCE, offset = 99,
|
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,
|
append(stateManager, producerId, epoch, RecordBatch.NO_SEQUENCE, offset = 100,
|
||||||
isTransactional = true, origin = AppendOrigin.Coordinator)
|
isTransactional = true, origin = AppendOrigin.COORDINATOR)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -994,7 +994,7 @@ class ProducerStateManagerTest {
|
||||||
when(batch.iterator).thenReturn(Collections.emptyIterator[Record])
|
when(batch.iterator).thenReturn(Collections.emptyIterator[Record])
|
||||||
|
|
||||||
// Appending the empty control batch should not throw and a new transaction shouldn't be started
|
// 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)
|
assertEquals(None, stateManager.lastEntry(producerId).get.currentTxnFirstOffset)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1102,7 +1102,7 @@ class ProducerStateManagerTest {
|
||||||
offset: Long,
|
offset: Long,
|
||||||
coordinatorEpoch: Int = 0,
|
coordinatorEpoch: Int = 0,
|
||||||
timestamp: Long = time.milliseconds()): Option[CompletedTxn] = {
|
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 endTxnMarker = new EndTransactionMarker(controlType, coordinatorEpoch)
|
||||||
val completedTxnOpt = producerAppendInfo.appendEndTxnMarker(endTxnMarker, producerEpoch, offset, timestamp)
|
val completedTxnOpt = producerAppendInfo.appendEndTxnMarker(endTxnMarker, producerEpoch, offset, timestamp)
|
||||||
mapping.update(producerAppendInfo)
|
mapping.update(producerAppendInfo)
|
||||||
|
@ -1118,7 +1118,7 @@ class ProducerStateManagerTest {
|
||||||
offset: Long,
|
offset: Long,
|
||||||
timestamp: Long = time.milliseconds(),
|
timestamp: Long = time.milliseconds(),
|
||||||
isTransactional: Boolean = false,
|
isTransactional: Boolean = false,
|
||||||
origin : AppendOrigin = AppendOrigin.Client): Unit = {
|
origin : AppendOrigin = AppendOrigin.CLIENT): Unit = {
|
||||||
val producerAppendInfo = stateManager.prepareUpdate(producerId, origin)
|
val producerAppendInfo = stateManager.prepareUpdate(producerId, origin)
|
||||||
producerAppendInfo.appendDataBatch(producerEpoch, seq, seq, timestamp,
|
producerAppendInfo.appendDataBatch(producerEpoch, seq, seq, timestamp,
|
||||||
LogOffsetMetadata(offset), offset, isTransactional)
|
LogOffsetMetadata(offset), offset, isTransactional)
|
||||||
|
|
|
@ -22,7 +22,7 @@ import java.nio.ByteBuffer
|
||||||
import java.nio.file.Files
|
import java.nio.file.Files
|
||||||
import java.util.concurrent.{Callable, ConcurrentHashMap, Executors}
|
import java.util.concurrent.{Callable, ConcurrentHashMap, Executors}
|
||||||
import java.util.{Optional, Properties}
|
import java.util.{Optional, Properties}
|
||||||
import kafka.common.{OffsetsOutOfOrderException, RecordValidationException, UnexpectedAppendOffsetException}
|
import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException}
|
||||||
import kafka.log.remote.RemoteLogManager
|
import kafka.log.remote.RemoteLogManager
|
||||||
import kafka.server.checkpoints.LeaderEpochCheckpointFile
|
import kafka.server.checkpoints.LeaderEpochCheckpointFile
|
||||||
import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache}
|
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.record._
|
||||||
import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse}
|
import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse}
|
||||||
import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils}
|
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.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig
|
||||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
|
@ -127,16 +127,16 @@ class UnifiedLogTest {
|
||||||
new SimpleRecord(mockTime.milliseconds, "c".getBytes, "value".getBytes)
|
new SimpleRecord(mockTime.milliseconds, "c".getBytes, "value".getBytes)
|
||||||
), baseOffset = offset, partitionLeaderEpoch = leaderEpoch)
|
), baseOffset = offset, partitionLeaderEpoch = leaderEpoch)
|
||||||
|
|
||||||
log.appendAsLeader(records(0), leaderEpoch, AppendOrigin.RaftLeader)
|
log.appendAsLeader(records(0), leaderEpoch, AppendOrigin.RAFT_LEADER)
|
||||||
assertEquals(0, log.logStartOffset)
|
assertEquals(0, log.logStartOffset)
|
||||||
assertEquals(3L, log.logEndOffset)
|
assertEquals(3L, log.logEndOffset)
|
||||||
|
|
||||||
// Since raft leader is responsible for assigning offsets, and the LogValidator is bypassed from the performance perspective,
|
// 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
|
// 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
|
// 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)
|
assertEquals(6, log.logEndOffset)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1788,22 +1788,22 @@ class UnifiedLogTest {
|
||||||
() => log.appendAsLeader(messageSetWithUnkeyedMessage, leaderEpoch = 0))
|
() => log.appendAsLeader(messageSetWithUnkeyedMessage, leaderEpoch = 0))
|
||||||
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
|
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
|
||||||
assertEquals(1, e.recordErrors.size)
|
assertEquals(1, e.recordErrors.size)
|
||||||
assertEquals(0, e.recordErrors.head.batchIndex)
|
assertEquals(0, e.recordErrors.get(0).batchIndex)
|
||||||
assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix))
|
assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix))
|
||||||
|
|
||||||
e = assertThrows(classOf[RecordValidationException],
|
e = assertThrows(classOf[RecordValidationException],
|
||||||
() => log.appendAsLeader(messageSetWithOneUnkeyedMessage, leaderEpoch = 0))
|
() => log.appendAsLeader(messageSetWithOneUnkeyedMessage, leaderEpoch = 0))
|
||||||
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
|
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
|
||||||
assertEquals(1, e.recordErrors.size)
|
assertEquals(1, e.recordErrors.size)
|
||||||
assertEquals(0, e.recordErrors.head.batchIndex)
|
assertEquals(0, e.recordErrors.get(0).batchIndex)
|
||||||
assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix))
|
assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix))
|
||||||
|
|
||||||
e = assertThrows(classOf[RecordValidationException],
|
e = assertThrows(classOf[RecordValidationException],
|
||||||
() => log.appendAsLeader(messageSetWithCompressedUnkeyedMessage, leaderEpoch = 0))
|
() => log.appendAsLeader(messageSetWithCompressedUnkeyedMessage, leaderEpoch = 0))
|
||||||
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
|
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
|
||||||
assertEquals(1, e.recordErrors.size)
|
assertEquals(1, e.recordErrors.size)
|
||||||
assertEquals(1, e.recordErrors.head.batchIndex) // batch index is 1
|
assertEquals(1, e.recordErrors.get(0).batchIndex) // batch index is 1
|
||||||
assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix))
|
assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix))
|
||||||
|
|
||||||
// check if metric for NoKeyCompactedTopicRecordsPerSec is logged
|
// check if metric for NoKeyCompactedTopicRecordsPerSec is logged
|
||||||
assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec}")), 1)
|
assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec}")), 1)
|
||||||
|
|
|
@ -28,7 +28,6 @@ import kafka.cluster.Broker
|
||||||
import kafka.controller.{ControllerContext, KafkaController}
|
import kafka.controller.{ControllerContext, KafkaController}
|
||||||
import kafka.coordinator.group._
|
import kafka.coordinator.group._
|
||||||
import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
|
import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
|
||||||
import kafka.log.AppendOrigin
|
|
||||||
import kafka.network.RequestChannel
|
import kafka.network.RequestChannel
|
||||||
import kafka.server.QuotaFactory.QuotaManagers
|
import kafka.server.QuotaFactory.QuotaManagers
|
||||||
import kafka.server.metadata.{ConfigRepository, KRaftMetadataCache, MockConfigRepository, ZkMetadataCache}
|
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.common.message.CreateTopicsResponseData.CreatableTopicResult
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
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.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1}
|
||||||
|
import org.apache.kafka.server.log.internals.AppendOrigin
|
||||||
|
|
||||||
class KafkaApisTest {
|
class KafkaApisTest {
|
||||||
private val requestChannel: RequestChannel = mock(classOf[RequestChannel])
|
private val requestChannel: RequestChannel = mock(classOf[RequestChannel])
|
||||||
|
@ -1645,7 +1645,7 @@ class KafkaApisTest {
|
||||||
when(replicaManager.appendRecords(anyLong,
|
when(replicaManager.appendRecords(anyLong,
|
||||||
anyShort,
|
anyShort,
|
||||||
ArgumentMatchers.eq(false),
|
ArgumentMatchers.eq(false),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Client),
|
ArgumentMatchers.eq(AppendOrigin.CLIENT),
|
||||||
any(),
|
any(),
|
||||||
responseCallback.capture(),
|
responseCallback.capture(),
|
||||||
any(),
|
any(),
|
||||||
|
@ -1786,7 +1786,7 @@ class KafkaApisTest {
|
||||||
when(replicaManager.appendRecords(anyLong,
|
when(replicaManager.appendRecords(anyLong,
|
||||||
anyShort,
|
anyShort,
|
||||||
ArgumentMatchers.eq(true),
|
ArgumentMatchers.eq(true),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
responseCallback.capture(),
|
responseCallback.capture(),
|
||||||
any(),
|
any(),
|
||||||
|
@ -1916,7 +1916,7 @@ class KafkaApisTest {
|
||||||
when(replicaManager.appendRecords(anyLong,
|
when(replicaManager.appendRecords(anyLong,
|
||||||
anyShort,
|
anyShort,
|
||||||
ArgumentMatchers.eq(true),
|
ArgumentMatchers.eq(true),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
responseCallback.capture(),
|
responseCallback.capture(),
|
||||||
any(),
|
any(),
|
||||||
|
@ -1948,7 +1948,7 @@ class KafkaApisTest {
|
||||||
verify(replicaManager).appendRecords(anyLong,
|
verify(replicaManager).appendRecords(anyLong,
|
||||||
anyShort,
|
anyShort,
|
||||||
ArgumentMatchers.eq(true),
|
ArgumentMatchers.eq(true),
|
||||||
ArgumentMatchers.eq(AppendOrigin.Coordinator),
|
ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
any(),
|
any(),
|
||||||
|
|
|
@ -21,7 +21,7 @@ import java.util
|
||||||
import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit}
|
import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit}
|
||||||
import java.util.{Optional, Properties}
|
import java.util.{Optional, Properties}
|
||||||
import kafka.api.LeaderAndIsr
|
import kafka.api.LeaderAndIsr
|
||||||
import kafka.log.{AppendOrigin, LogConfig}
|
import kafka.log.LogConfig
|
||||||
import kafka.server.metadata.KRaftMetadataCache
|
import kafka.server.metadata.KRaftMetadataCache
|
||||||
import kafka.server.metadata.MockConfigRepository
|
import kafka.server.metadata.MockConfigRepository
|
||||||
import kafka.utils.TestUtils.waitUntilTrue
|
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.image.{MetadataDelta, MetadataImage}
|
||||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||||
import org.apache.kafka.metadata.PartitionRegistration
|
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.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, Test}
|
import org.junit.jupiter.api.{AfterEach, Test}
|
||||||
import org.mockito.Mockito
|
import org.mockito.Mockito
|
||||||
|
@ -283,7 +284,7 @@ class ReplicaManagerConcurrencyTest {
|
||||||
timeout = 30000,
|
timeout = 30000,
|
||||||
requiredAcks = (-1).toShort,
|
requiredAcks = (-1).toShort,
|
||||||
internalTopicsAllowed = false,
|
internalTopicsAllowed = false,
|
||||||
origin = AppendOrigin.Client,
|
origin = AppendOrigin.CLIENT,
|
||||||
entriesPerPartition = collection.Map(topicPartition -> TestUtils.records(records)),
|
entriesPerPartition = collection.Map(topicPartition -> TestUtils.records(records)),
|
||||||
responseCallback = produceCallback
|
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.LeaderConstants.NO_LEADER
|
||||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||||
import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
|
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.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
|
@ -178,7 +179,7 @@ class ReplicaManagerTest {
|
||||||
timeout = 0,
|
timeout = 0,
|
||||||
requiredAcks = 3,
|
requiredAcks = 3,
|
||||||
internalTopicsAllowed = false,
|
internalTopicsAllowed = false,
|
||||||
origin = AppendOrigin.Client,
|
origin = AppendOrigin.CLIENT,
|
||||||
entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(CompressionType.NONE,
|
entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(CompressionType.NONE,
|
||||||
new SimpleRecord("first message".getBytes))),
|
new SimpleRecord("first message".getBytes))),
|
||||||
responseCallback = callback)
|
responseCallback = callback)
|
||||||
|
@ -456,7 +457,7 @@ class ReplicaManagerTest {
|
||||||
val abortTxnMarker = new EndTransactionMarker(ControlRecordType.ABORT, 0)
|
val abortTxnMarker = new EndTransactionMarker(ControlRecordType.ABORT, 0)
|
||||||
val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, abortTxnMarker)
|
val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, abortTxnMarker)
|
||||||
appendRecords(replicaManager, new TopicPartition(topic, 0),
|
appendRecords(replicaManager, new TopicPartition(topic, 0),
|
||||||
abortRecordBatch, origin = AppendOrigin.Coordinator).onFire { response =>
|
abortRecordBatch, origin = AppendOrigin.COORDINATOR).onFire { response =>
|
||||||
assertEquals(Errors.NONE, response.error)
|
assertEquals(Errors.NONE, response.error)
|
||||||
}
|
}
|
||||||
assertLateTransactionCount(Some(0))
|
assertLateTransactionCount(Some(0))
|
||||||
|
@ -550,7 +551,7 @@ class ReplicaManagerTest {
|
||||||
val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0)
|
val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0)
|
||||||
val commitRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker)
|
val commitRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker)
|
||||||
appendRecords(replicaManager, new TopicPartition(topic, 0), commitRecordBatch,
|
appendRecords(replicaManager, new TopicPartition(topic, 0), commitRecordBatch,
|
||||||
origin = AppendOrigin.Coordinator)
|
origin = AppendOrigin.COORDINATOR)
|
||||||
.onFire { response => assertEquals(Errors.NONE, response.error) }
|
.onFire { response => assertEquals(Errors.NONE, response.error) }
|
||||||
|
|
||||||
// the LSO has advanced, but the appended commit marker has not been replicated, so
|
// 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 endTxnMarker = new EndTransactionMarker(ControlRecordType.ABORT, 0)
|
||||||
val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker)
|
val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker)
|
||||||
appendRecords(replicaManager, new TopicPartition(topic, 0), abortRecordBatch,
|
appendRecords(replicaManager, new TopicPartition(topic, 0), abortRecordBatch,
|
||||||
origin = AppendOrigin.Coordinator)
|
origin = AppendOrigin.COORDINATOR)
|
||||||
.onFire { response => assertEquals(Errors.NONE, response.error) }
|
.onFire { response => assertEquals(Errors.NONE, response.error) }
|
||||||
|
|
||||||
// fetch as follower to advance the high watermark
|
// 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 simpleRecords = Seq(new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))
|
||||||
val appendResult = appendRecords(replicaManager, tp0,
|
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
|
// Increment the hw in the leader by fetching from the last offset
|
||||||
val fetchOffset = simpleRecords.size
|
val fetchOffset = simpleRecords.size
|
||||||
|
@ -1929,7 +1930,7 @@ class ReplicaManagerTest {
|
||||||
timeout = 10,
|
timeout = 10,
|
||||||
requiredAcks = -1,
|
requiredAcks = -1,
|
||||||
internalTopicsAllowed = false,
|
internalTopicsAllowed = false,
|
||||||
origin = AppendOrigin.Client,
|
origin = AppendOrigin.CLIENT,
|
||||||
entriesPerPartition = Map(topicPartition -> records),
|
entriesPerPartition = Map(topicPartition -> records),
|
||||||
responseCallback = callback
|
responseCallback = callback
|
||||||
)
|
)
|
||||||
|
@ -2149,7 +2150,7 @@ class ReplicaManagerTest {
|
||||||
private def appendRecords(replicaManager: ReplicaManager,
|
private def appendRecords(replicaManager: ReplicaManager,
|
||||||
partition: TopicPartition,
|
partition: TopicPartition,
|
||||||
records: MemoryRecords,
|
records: MemoryRecords,
|
||||||
origin: AppendOrigin = AppendOrigin.Client,
|
origin: AppendOrigin = AppendOrigin.CLIENT,
|
||||||
requiredAcks: Short = -1): CallbackResult[PartitionResponse] = {
|
requiredAcks: Short = -1): CallbackResult[PartitionResponse] = {
|
||||||
val result = new CallbackResult[PartitionResponse]()
|
val result = new CallbackResult[PartitionResponse]()
|
||||||
def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = {
|
def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = {
|
||||||
|
@ -2786,7 +2787,7 @@ class ReplicaManagerTest {
|
||||||
val batch = TestUtils.records(records = List(
|
val batch = TestUtils.records(records = List(
|
||||||
new SimpleRecord(10, "k1".getBytes, "v1".getBytes),
|
new SimpleRecord(10, "k1".getBytes, "v1".getBytes),
|
||||||
new SimpleRecord(11, "k2".getBytes, "v2".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.updateHighWatermark(2L)
|
||||||
partition.log.get.maybeIncrementLogStartOffset(1L, LeaderOffsetIncremented)
|
partition.log.get.maybeIncrementLogStartOffset(1L, LeaderOffsetIncremented)
|
||||||
replicaManager.logManager.checkpointLogRecoveryOffsets()
|
replicaManager.logManager.checkpointLogRecoveryOffsets()
|
||||||
|
|
|
@ -21,8 +21,7 @@ import java.io.{ByteArrayOutputStream, File, PrintWriter}
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.util
|
import java.util
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
import kafka.log.{Defaults, LogConfig, LogTestUtils, ProducerStateManagerConfig, UnifiedLog}
|
||||||
import kafka.log.{AppendOrigin, Defaults, LogConfig, LogTestUtils, ProducerStateManagerConfig, UnifiedLog}
|
|
||||||
import kafka.raft.{KafkaMetadataLog, MetadataLogConfig}
|
import kafka.raft.{KafkaMetadataLog, MetadataLogConfig}
|
||||||
import kafka.server.{BrokerTopicStats, FetchLogEnd, KafkaRaftServer, LogDirFailureChannel}
|
import kafka.server.{BrokerTopicStats, FetchLogEnd, KafkaRaftServer, LogDirFailureChannel}
|
||||||
import kafka.tools.DumpLogSegments.TimeIndexDumpErrors
|
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.metadata.MetadataRecordSerde
|
||||||
import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch}
|
import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch}
|
||||||
import org.apache.kafka.server.common.ApiMessageAndVersion
|
import org.apache.kafka.server.common.ApiMessageAndVersion
|
||||||
|
import org.apache.kafka.server.log.internals.AppendOrigin
|
||||||
import org.apache.kafka.snapshot.RecordsSnapshotWriter
|
import org.apache.kafka.snapshot.RecordsSnapshotWriter
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||||
|
@ -136,7 +136,7 @@ class DumpLogSegmentsTest {
|
||||||
|
|
||||||
log.appendAsLeader(MemoryRecords.withEndTransactionMarker(98323L, 99.toShort,
|
log.appendAsLeader(MemoryRecords.withEndTransactionMarker(98323L, 99.toShort,
|
||||||
new EndTransactionMarker(ControlRecordType.COMMIT, 100)
|
new EndTransactionMarker(ControlRecordType.COMMIT, 100)
|
||||||
), origin = AppendOrigin.Coordinator, leaderEpoch = 7)
|
), origin = AppendOrigin.COORDINATOR, leaderEpoch = 7)
|
||||||
|
|
||||||
assertDumpLogRecordMetadata()
|
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"/>
|
<Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
|
||||||
</Match>
|
</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>
|
<Match>
|
||||||
<!-- Suppress warnings related to jmh generated code -->
|
<!-- Suppress warnings related to jmh generated code -->
|
||||||
<Package name="org.apache.kafka.jmh.acl.generated"/>
|
<Package name="org.apache.kafka.jmh.acl.generated"/>
|
||||||
|
|
|
@ -16,6 +16,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.jmh.record;
|
package org.apache.kafka.jmh.record;
|
||||||
|
|
||||||
|
import kafka.log.UnifiedLog;
|
||||||
import kafka.server.BrokerTopicStats;
|
import kafka.server.BrokerTopicStats;
|
||||||
import kafka.server.RequestLocal;
|
import kafka.server.RequestLocal;
|
||||||
import org.apache.kafka.common.header.Header;
|
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.Record;
|
||||||
import org.apache.kafka.common.record.RecordBatch;
|
import org.apache.kafka.common.record.RecordBatch;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
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.Param;
|
||||||
import org.openjdk.jmh.annotations.Scope;
|
import org.openjdk.jmh.annotations.Scope;
|
||||||
import org.openjdk.jmh.annotations.Setup;
|
import org.openjdk.jmh.annotations.Setup;
|
||||||
|
@ -75,7 +77,8 @@ public abstract class BaseRecordBatchBenchmark {
|
||||||
// Used by measureVariableBatchSize
|
// Used by measureVariableBatchSize
|
||||||
ByteBuffer[] batchBuffers;
|
ByteBuffer[] batchBuffers;
|
||||||
RequestLocal requestLocal;
|
RequestLocal requestLocal;
|
||||||
final BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
|
LogValidator.MetricsRecorder validatorMetricsRecorder = UnifiedLog.newValidatorMetricsRecorder(
|
||||||
|
new BrokerTopicStats().allTopicsStats());
|
||||||
|
|
||||||
@Setup
|
@Setup
|
||||||
public void init() {
|
public void init() {
|
||||||
|
|
|
@ -16,15 +16,15 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.jmh.record;
|
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.TopicPartition;
|
||||||
import org.apache.kafka.common.record.CompressionType;
|
import org.apache.kafka.common.record.CompressionType;
|
||||||
import org.apache.kafka.common.record.MemoryRecords;
|
import org.apache.kafka.common.record.MemoryRecords;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
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.common.utils.Time;
|
||||||
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.log.internals.LogValidator;
|
||||||
import org.openjdk.jmh.annotations.Benchmark;
|
import org.openjdk.jmh.annotations.Benchmark;
|
||||||
import org.openjdk.jmh.annotations.Fork;
|
import org.openjdk.jmh.annotations.Fork;
|
||||||
import org.openjdk.jmh.annotations.Measurement;
|
import org.openjdk.jmh.annotations.Measurement;
|
||||||
|
@ -51,14 +51,11 @@ public class CompressedRecordBatchValidationBenchmark extends BaseRecordBatchBen
|
||||||
@Benchmark
|
@Benchmark
|
||||||
public void measureValidateMessagesAndAssignOffsetsCompressed(Blackhole bh) {
|
public void measureValidateMessagesAndAssignOffsetsCompressed(Blackhole bh) {
|
||||||
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
||||||
LogValidator.validateMessagesAndAssignOffsetsCompressed(records, new TopicPartition("a", 0),
|
new LogValidator(records, new TopicPartition("a", 0),
|
||||||
new LongRef(startingOffset), Time.SYSTEM, System.currentTimeMillis(),
|
Time.SYSTEM, compressionType, compressionType, false, messageVersion,
|
||||||
compressionType,
|
TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
|
||||||
compressionType,
|
MetadataVersion.latest()
|
||||||
false, messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0,
|
).validateMessagesAndAssignOffsetsCompressed(PrimitiveRef.ofLong(startingOffset),
|
||||||
new AppendOrigin.Client$(),
|
validatorMetricsRecorder, requestLocal.bufferSupplier());
|
||||||
MetadataVersion.latest(),
|
|
||||||
brokerTopicStats,
|
|
||||||
requestLocal);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,13 +16,15 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.jmh.record;
|
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.TopicPartition;
|
||||||
import org.apache.kafka.common.record.CompressionType;
|
import org.apache.kafka.common.record.CompressionType;
|
||||||
import org.apache.kafka.common.record.MemoryRecords;
|
import org.apache.kafka.common.record.MemoryRecords;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
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.Benchmark;
|
||||||
import org.openjdk.jmh.annotations.Fork;
|
import org.openjdk.jmh.annotations.Fork;
|
||||||
import org.openjdk.jmh.annotations.Measurement;
|
import org.openjdk.jmh.annotations.Measurement;
|
||||||
|
@ -45,9 +47,10 @@ public class UncompressedRecordBatchValidationBenchmark extends BaseRecordBatchB
|
||||||
@Benchmark
|
@Benchmark
|
||||||
public void measureAssignOffsetsNonCompressed(Blackhole bh) {
|
public void measureAssignOffsetsNonCompressed(Blackhole bh) {
|
||||||
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
||||||
LogValidator.assignOffsetsNonCompressed(records, new TopicPartition("a", 0),
|
new LogValidator(records, new TopicPartition("a", 0),
|
||||||
new LongRef(startingOffset), System.currentTimeMillis(), false,
|
Time.SYSTEM, CompressionType.NONE, CompressionType.NONE, false,
|
||||||
TimestampType.CREATE_TIME, Long.MAX_VALUE, 0,
|
messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
|
||||||
new AppendOrigin.Client$(), messageVersion, brokerTopicStats);
|
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