Browse Source

KAFKA-7920; Do not permit zstd produce requests until IBP is updated to 2.1 (#6256)

Fail produce requests using zstd until the inter.broker.protocol.version is large enough that replicas are ensured to support it. Otherwise, followers receive the `UNSUPPORTED_COMPRESSION_TYPE` when fetching zstd data and ISRs shrink.

Reviewers: Jason Gustafson <jason@confluent.io>
pull/6297/head
Lee Dongjin 6 years ago committed by Jason Gustafson
parent
commit
71a7219dfd
  1. 2
      clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
  2. 8
      core/src/main/scala/kafka/cluster/Partition.scala
  3. 16
      core/src/main/scala/kafka/log/Log.scala
  4. 18
      core/src/main/scala/kafka/log/LogValidator.scala
  5. 9
      core/src/main/scala/kafka/server/KafkaConfig.scala
  6. 1
      core/src/main/scala/kafka/server/ReplicaManager.scala
  7. 5
      core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
  8. 136
      core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
  9. 4
      core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala
  10. 7
      core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala

2
clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java

@ -402,7 +402,7 @@ public class ProduceRequest extends AbstractRequest { @@ -402,7 +402,7 @@ public class ProduceRequest extends AbstractRequest {
throw new InvalidRecordException("Produce requests with version " + version + " are only allowed to " +
"contain record batches with magic version 2");
if (version < 7 && entry.compressionType() == CompressionType.ZSTD) {
throw new UnsupportedCompressionTypeException("Produce requests with version " + version + " are note allowed to " +
throw new UnsupportedCompressionTypeException("Produce requests with version " + version + " are not allowed to " +
"use ZStandard compression");
}

8
core/src/main/scala/kafka/cluster/Partition.scala

@ -20,7 +20,7 @@ import java.util.Optional @@ -20,7 +20,7 @@ import java.util.Optional
import java.util.concurrent.locks.ReentrantReadWriteLock
import com.yammer.metrics.core.Gauge
import kafka.api.{LeaderAndIsr, Request}
import kafka.api.{ApiVersion, LeaderAndIsr, Request}
import kafka.common.UnexpectedAppendOffsetException
import kafka.controller.KafkaController
import kafka.log._
@ -49,6 +49,7 @@ object Partition { @@ -49,6 +49,7 @@ object Partition {
new Partition(topicPartition,
isOffline = false,
replicaLagTimeMaxMs = replicaManager.config.replicaLagTimeMaxMs,
interBrokerProtocolVersion = replicaManager.config.interBrokerProtocolVersion,
localBrokerId = replicaManager.config.brokerId,
time = time,
replicaManager = replicaManager,
@ -63,6 +64,7 @@ object Partition { @@ -63,6 +64,7 @@ object Partition {
class Partition(val topicPartition: TopicPartition,
val isOffline: Boolean,
private val replicaLagTimeMaxMs: Long,
private val interBrokerProtocolVersion: ApiVersion,
private val localBrokerId: Int,
private val time: Time,
private val replicaManager: ReplicaManager,
@ -748,7 +750,9 @@ class Partition(val topicPartition: TopicPartition, @@ -748,7 +750,9 @@ class Partition(val topicPartition: TopicPartition,
s"is insufficient to satisfy the min.isr requirement of $minIsr for partition $topicPartition")
}
val info = log.appendAsLeader(records, leaderEpoch = this.leaderEpoch, isFromClient)
val info = log.appendAsLeader(records, leaderEpoch = this.leaderEpoch, isFromClient,
interBrokerProtocolVersion)
// we may need to increment high watermark since ISR could be down to 1
(info, maybeIncrementLeaderHW(leaderReplica))

16
core/src/main/scala/kafka/log/Log.scala

@ -28,7 +28,7 @@ import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap, Time @@ -28,7 +28,7 @@ import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap, Time
import java.util.regex.Pattern
import com.yammer.metrics.core.Gauge
import kafka.api.KAFKA_0_10_0_IV0
import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0}
import kafka.common.{LogSegmentOffsetOverflowException, LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException}
import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
import kafka.metrics.KafkaMetricsGroup
@ -787,11 +787,13 @@ class Log(@volatile var dir: File, @@ -787,11 +787,13 @@ class Log(@volatile var dir: File,
*
* @param records The records to append
* @param isFromClient Whether or not this append is from a producer
* @param interBrokerProtocolVersion Inter-broker message protocol version
* @throws KafkaStorageException If the append fails due to an I/O error.
* @return Information about the appended messages including the first and last offset.
*/
def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, isFromClient: Boolean = true): LogAppendInfo = {
append(records, isFromClient, assignOffsets = true, leaderEpoch)
def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, isFromClient: Boolean = true,
interBrokerProtocolVersion: ApiVersion = ApiVersion.latestVersion): LogAppendInfo = {
append(records, isFromClient, interBrokerProtocolVersion, assignOffsets = true, leaderEpoch)
}
/**
@ -802,7 +804,7 @@ class Log(@volatile var dir: File, @@ -802,7 +804,7 @@ class Log(@volatile var dir: File,
* @return Information about the appended messages including the first and last offset.
*/
def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
append(records, isFromClient = false, assignOffsets = false, leaderEpoch = -1)
append(records, isFromClient = false, interBrokerProtocolVersion = ApiVersion.latestVersion, assignOffsets = false, leaderEpoch = -1)
}
/**
@ -813,6 +815,7 @@ class Log(@volatile var dir: File, @@ -813,6 +815,7 @@ class Log(@volatile var dir: File,
*
* @param records The log records to append
* @param isFromClient Whether or not this append is from a producer
* @param interBrokerProtocolVersion Inter-broker message protocol version
* @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given
* @param leaderEpoch The partition's leader epoch which will be applied to messages when offsets are assigned on the leader
* @throws KafkaStorageException If the append fails due to an I/O error.
@ -820,7 +823,7 @@ class Log(@volatile var dir: File, @@ -820,7 +823,7 @@ class Log(@volatile var dir: File,
* @throws UnexpectedAppendOffsetException If the first or last offset in append is less than next offset
* @return Information about the appended messages including the first and last offset.
*/
private def append(records: MemoryRecords, isFromClient: Boolean, assignOffsets: Boolean, leaderEpoch: Int): LogAppendInfo = {
private def append(records: MemoryRecords, isFromClient: Boolean, interBrokerProtocolVersion: ApiVersion, assignOffsets: Boolean, leaderEpoch: Int): LogAppendInfo = {
maybeHandleIOException(s"Error while appending records to $topicPartition in dir ${dir.getParent}") {
val appendInfo = analyzeAndValidateRecords(records, isFromClient = isFromClient)
@ -851,7 +854,8 @@ class Log(@volatile var dir: File, @@ -851,7 +854,8 @@ class Log(@volatile var dir: File,
config.messageTimestampType,
config.messageTimestampDifferenceMaxMs,
leaderEpoch,
isFromClient)
isFromClient,
interBrokerProtocolVersion)
} catch {
case e: IOException =>
throw new KafkaException(s"Error validating messages while appending to log $name", e)

18
core/src/main/scala/kafka/log/LogValidator.scala

@ -18,10 +18,11 @@ package kafka.log @@ -18,10 +18,11 @@ package kafka.log
import java.nio.ByteBuffer
import kafka.api.{ApiVersion, KAFKA_2_1_IV0}
import kafka.common.LongRef
import kafka.message.{CompressionCodec, NoCompressionCodec}
import kafka.message.{CompressionCodec, NoCompressionCodec, ZStdCompressionCodec}
import kafka.utils.Logging
import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.record.{AbstractRecords, CompressionType, InvalidRecordException, MemoryRecords, Record, RecordBatch, RecordConversionStats, TimestampType}
import org.apache.kafka.common.utils.Time
@ -56,7 +57,8 @@ private[kafka] object LogValidator extends Logging { @@ -56,7 +57,8 @@ private[kafka] object LogValidator extends Logging {
timestampType: TimestampType,
timestampDiffMaxMs: Long,
partitionLeaderEpoch: Int,
isFromClient: Boolean): ValidationAndOffsetAssignResult = {
isFromClient: Boolean,
interBrokerProtocolVersion: ApiVersion): ValidationAndOffsetAssignResult = {
if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
// check the magic value
if (!records.hasMatchingMagic(magic))
@ -68,7 +70,7 @@ private[kafka] object LogValidator extends Logging { @@ -68,7 +70,7 @@ private[kafka] object LogValidator extends Logging {
partitionLeaderEpoch, isFromClient, magic)
} else {
validateMessagesAndAssignOffsetsCompressed(records, offsetCounter, time, now, sourceCodec, targetCodec, compactedTopic,
magic, timestampType, timestampDiffMaxMs, partitionLeaderEpoch, isFromClient)
magic, timestampType, timestampDiffMaxMs, partitionLeaderEpoch, isFromClient, interBrokerProtocolVersion)
}
}
@ -245,8 +247,8 @@ private[kafka] object LogValidator extends Logging { @@ -245,8 +247,8 @@ private[kafka] object LogValidator extends Logging {
timestampType: TimestampType,
timestampDiffMaxMs: Long,
partitionLeaderEpoch: Int,
isFromClient: Boolean): ValidationAndOffsetAssignResult = {
isFromClient: Boolean,
interBrokerProtocolVersion: ApiVersion): ValidationAndOffsetAssignResult = {
// No in place assignment situation 1 and 2
var inPlaceAssignment = sourceCodec == targetCodec && toMagic > RecordBatch.MAGIC_VALUE_V0
@ -265,10 +267,12 @@ private[kafka] object LogValidator extends Logging { @@ -265,10 +267,12 @@ private[kafka] object LogValidator extends Logging {
inPlaceAssignment = true
for (record <- batch.asScala) {
validateRecord(batch, record, now, timestampType, timestampDiffMaxMs, compactedTopic)
if (sourceCodec != NoCompressionCodec && record.isCompressed)
throw new InvalidRecordException("Compressed outer record should not have an inner record with a " +
s"compression attribute set: $record")
if (targetCodec == ZStdCompressionCodec && interBrokerProtocolVersion < KAFKA_2_1_IV0)
throw new UnsupportedCompressionTypeException("Produce requests to inter.broker.protocol.version < 2.1 broker " + "are not allowed to use ZStandard compression")
validateRecord(batch, record, now, timestampType, timestampDiffMaxMs, compactedTopic)
uncompressedSizeInBytes += record.sizeInBytes()
if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && toMagic > RecordBatch.MAGIC_VALUE_V0) {

9
core/src/main/scala/kafka/server/KafkaConfig.scala

@ -20,11 +20,11 @@ package kafka.server @@ -20,11 +20,11 @@ package kafka.server
import java.util
import java.util.{Collections, Properties}
import kafka.api.{ApiVersion, ApiVersionValidator, KAFKA_0_10_0_IV1}
import kafka.api.{ApiVersion, ApiVersionValidator, KAFKA_0_10_0_IV1, KAFKA_2_1_IV0}
import kafka.cluster.EndPoint
import kafka.coordinator.group.OffsetConfig
import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager}
import kafka.message.{BrokerCompressionCodec, CompressionCodec}
import kafka.message.{BrokerCompressionCodec, CompressionCodec, ZStdCompressionCodec}
import kafka.utils.CoreUtils
import kafka.utils.Implicits._
import org.apache.kafka.clients.CommonClientConfigs
@ -1486,6 +1486,11 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO @@ -1486,6 +1486,11 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO
s"log.message.format.version $logMessageFormatVersionString can only be used when inter.broker.protocol.version " +
s"is set to version ${ApiVersion.minSupportedFor(recordVersion).shortVersion} or higher")
if (offsetsTopicCompressionCodec == ZStdCompressionCodec)
require(interBrokerProtocolVersion.recordVersion.value >= KAFKA_2_1_IV0.recordVersion.value,
"offsets.topic.compression.codec zstd can only be used when inter.broker.protocol.version " +
s"is set to version ${KAFKA_2_1_IV0.shortVersion} or higher")
val interBrokerUsesSasl = interBrokerSecurityProtocol == SecurityProtocol.SASL_PLAINTEXT || interBrokerSecurityProtocol == SecurityProtocol.SASL_SSL
require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM,
s"Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to $interBrokerProtocolVersionString")

1
core/src/main/scala/kafka/server/ReplicaManager.scala

@ -125,6 +125,7 @@ object ReplicaManager { @@ -125,6 +125,7 @@ object ReplicaManager {
val OfflinePartition: Partition = new Partition(new TopicPartition("", -1),
isOffline = true,
replicaLagTimeMaxMs = 0L,
interBrokerProtocolVersion = ApiVersion.latestVersion,
localBrokerId = -1,
time = null,
replicaManager = null,

5
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala

@ -22,7 +22,7 @@ import java.util.{Optional, Properties} @@ -22,7 +22,7 @@ import java.util.{Optional, Properties}
import java.util.concurrent.{CountDownLatch, Executors, TimeUnit, TimeoutException}
import java.util.concurrent.atomic.AtomicBoolean
import kafka.api.Request
import kafka.api.{ApiVersion, Request}
import kafka.common.UnexpectedAppendOffsetException
import kafka.log.{Defaults => _, _}
import kafka.server._
@ -590,6 +590,7 @@ class PartitionTest { @@ -590,6 +590,7 @@ class PartitionTest {
val partition = new Partition(topicPartition,
isOffline = false,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
interBrokerProtocolVersion = ApiVersion.latestVersion,
localBrokerId = brokerId,
time,
replicaManager,
@ -685,6 +686,7 @@ class PartitionTest { @@ -685,6 +686,7 @@ class PartitionTest {
val partition = new Partition(topicPartition,
isOffline = false,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
interBrokerProtocolVersion = ApiVersion.latestVersion,
localBrokerId = brokerId,
time,
replicaManager,
@ -897,6 +899,7 @@ class PartitionTest { @@ -897,6 +899,7 @@ class PartitionTest {
val partition = new Partition(tp,
isOffline = false,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs,
interBrokerProtocolVersion = ApiVersion.latestVersion,
localBrokerId = brokerId,
time,
replicaManager,

136
core/src/test/scala/unit/kafka/log/LogValidatorTest.scala

@ -19,9 +19,10 @@ package kafka.log @@ -19,9 +19,10 @@ package kafka.log
import java.nio.ByteBuffer
import java.util.concurrent.TimeUnit
import kafka.api.{ApiVersion, KAFKA_2_0_IV1}
import kafka.common.LongRef
import kafka.message.{CompressionCodec, DefaultCompressionCodec, GZIPCompressionCodec, NoCompressionCodec, SnappyCompressionCodec}
import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedForMessageFormatException}
import kafka.message._
import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Time
import org.apache.kafka.test.TestUtils
@ -55,7 +56,8 @@ class LogValidatorTest { @@ -55,7 +56,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatedResults.validatedRecords
assertEquals("message set size should not change", records.records.asScala.size, validatedRecords.records.asScala.size)
validatedRecords.batches.asScala.foreach(batch => validateLogAppendTime(now, 1234L, batch))
@ -92,7 +94,8 @@ class LogValidatorTest { @@ -92,7 +94,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatedResults.validatedRecords
assertEquals("message set size should not change", records.records.asScala.size, validatedRecords.records.asScala.size)
@ -133,7 +136,8 @@ class LogValidatorTest { @@ -133,7 +136,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatedResults.validatedRecords
assertEquals("message set size should not change", records.records.asScala.size,
@ -190,7 +194,8 @@ class LogValidatorTest { @@ -190,7 +194,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
@Test
@ -231,7 +236,8 @@ class LogValidatorTest { @@ -231,7 +236,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = partitionLeaderEpoch,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatingResults.validatedRecords
var i = 0
@ -297,7 +303,8 @@ class LogValidatorTest { @@ -297,7 +303,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = partitionLeaderEpoch,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatingResults.validatedRecords
var i = 0
@ -347,7 +354,8 @@ class LogValidatorTest { @@ -347,7 +354,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatedResults.validatedRecords
for (batch <- validatedRecords.batches.asScala) {
@ -388,7 +396,8 @@ class LogValidatorTest { @@ -388,7 +396,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatedResults.validatedRecords
for (batch <- validatedRecords.batches.asScala) {
@ -442,7 +451,8 @@ class LogValidatorTest { @@ -442,7 +451,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = partitionLeaderEpoch,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val validatedRecords = validatedResults.validatedRecords
var i = 0
@ -492,7 +502,8 @@ class LogValidatorTest { @@ -492,7 +502,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
@Test(expected = classOf[InvalidTimestampException])
@ -512,7 +523,8 @@ class LogValidatorTest { @@ -512,7 +523,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
@Test(expected = classOf[InvalidTimestampException])
@ -532,7 +544,8 @@ class LogValidatorTest { @@ -532,7 +544,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
@Test(expected = classOf[InvalidTimestampException])
@ -552,7 +565,8 @@ class LogValidatorTest { @@ -552,7 +565,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
@Test
@ -571,7 +585,8 @@ class LogValidatorTest { @@ -571,7 +585,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -590,7 +605,8 @@ class LogValidatorTest { @@ -590,7 +605,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -610,7 +626,8 @@ class LogValidatorTest { @@ -610,7 +626,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords
checkOffsets(messageWithOffset, offset)
}
@ -631,7 +648,8 @@ class LogValidatorTest { @@ -631,7 +648,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords
checkOffsets(messageWithOffset, offset)
}
@ -653,7 +671,8 @@ class LogValidatorTest { @@ -653,7 +671,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords
checkOffsets(compressedMessagesWithOffset, offset)
}
@ -675,7 +694,8 @@ class LogValidatorTest { @@ -675,7 +694,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords
checkOffsets(compressedMessagesWithOffset, offset)
}
@ -695,7 +715,8 @@ class LogValidatorTest { @@ -695,7 +715,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
checkOffsets(validatedResults.validatedRecords, offset)
verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records,
compressed = false)
@ -717,7 +738,8 @@ class LogValidatorTest { @@ -717,7 +738,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
checkOffsets(validatedResults.validatedRecords, offset)
verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records,
compressed = false)
@ -739,7 +761,8 @@ class LogValidatorTest { @@ -739,7 +761,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
checkOffsets(validatedResults.validatedRecords, offset)
verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records,
compressed = true)
@ -761,7 +784,8 @@ class LogValidatorTest { @@ -761,7 +784,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
checkOffsets(validatedResults.validatedRecords, offset)
verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records,
compressed = true)
@ -783,7 +807,8 @@ class LogValidatorTest { @@ -783,7 +807,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
@Test
@ -802,7 +827,8 @@ class LogValidatorTest { @@ -802,7 +827,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = false)
isFromClient = false,
interBrokerProtocolVersion = ApiVersion.latestVersion)
val batches = TestUtils.toList(result.validatedRecords.batches)
assertEquals(1, batches.size)
val batch = batches.get(0)
@ -826,7 +852,8 @@ class LogValidatorTest { @@ -826,7 +852,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -846,7 +873,8 @@ class LogValidatorTest { @@ -846,7 +873,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -865,7 +893,8 @@ class LogValidatorTest { @@ -865,7 +893,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -884,7 +913,8 @@ class LogValidatorTest { @@ -884,7 +913,8 @@ class LogValidatorTest {
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -904,7 +934,8 @@ class LogValidatorTest { @@ -904,7 +934,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -924,7 +955,8 @@ class LogValidatorTest { @@ -924,7 +955,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test(expected = classOf[UnsupportedForMessageFormatException])
@ -946,7 +978,8 @@ class LogValidatorTest { @@ -946,7 +978,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test(expected = classOf[UnsupportedForMessageFormatException])
@ -968,7 +1001,8 @@ class LogValidatorTest { @@ -968,7 +1001,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -988,7 +1022,8 @@ class LogValidatorTest { @@ -988,7 +1022,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test
@ -1008,7 +1043,8 @@ class LogValidatorTest { @@ -1008,7 +1043,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true).validatedRecords, offset)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion).validatedRecords, offset)
}
@Test(expected = classOf[InvalidRecordException])
@ -1026,7 +1062,8 @@ class LogValidatorTest { @@ -1026,7 +1062,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
@Test(expected = classOf[InvalidRecordException])
@ -1034,6 +1071,26 @@ class LogValidatorTest { @@ -1034,6 +1071,26 @@ class LogValidatorTest {
testBatchWithoutRecordsNotAllowed(DefaultCompressionCodec, DefaultCompressionCodec)
}
@Test(expected = classOf[UnsupportedCompressionTypeException])
def testZStdCompressedWithUnavailableIBPVersion(): Unit = {
val now = System.currentTimeMillis()
// The timestamps should be overwritten
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE)
LogValidator.validateMessagesAndAssignOffsets(records,
offsetCounter = new LongRef(0),
time= time,
now = now,
sourceCodec = NoCompressionCodec,
targetCodec = ZStdCompressionCodec,
compactedTopic = false,
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true,
interBrokerProtocolVersion = KAFKA_2_0_IV1)
}
@Test(expected = classOf[InvalidRecordException])
def testUncompressedBatchWithoutRecordsNotAllowed(): Unit = {
testBatchWithoutRecordsNotAllowed(NoCompressionCodec, NoCompressionCodec)
@ -1065,7 +1122,8 @@ class LogValidatorTest { @@ -1065,7 +1122,8 @@ class LogValidatorTest {
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
isFromClient = true)
isFromClient = true,
interBrokerProtocolVersion = ApiVersion.latestVersion)
}
private def createRecords(magicValue: Byte,

4
core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala

@ -23,10 +23,10 @@ import java.util.Properties @@ -23,10 +23,10 @@ import java.util.Properties
import kafka.network.SocketServer
import kafka.utils.TestUtils
import org.apache.kafka.common.message.CreateTopicsRequestData
import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableReplicaAssignment, CreatableReplicaAssignmentSet, CreatableTopic, CreatableTopicSet, CreateableTopicConfig, CreateableTopicConfigSet}
import org.apache.kafka.common.message.CreateTopicsRequestData._
import org.apache.kafka.common.protocol.types.Struct
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{ApiError, CreateTopicsRequest, CreateTopicsResponse, MetadataRequest, MetadataResponse}
import org.apache.kafka.common.requests._
import org.junit.Assert.{assertEquals, assertFalse, assertNotNull, assertTrue}
import scala.collection.JavaConverters._

7
core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala

@ -18,18 +18,17 @@ package kafka.server @@ -18,18 +18,17 @@ package kafka.server
import java.util.Optional
import kafka.cluster.{BrokerEndPoint, Replica}
import kafka.cluster.{BrokerEndPoint, Partition, Replica}
import kafka.log.LogManager
import kafka.cluster.Partition
import kafka.server.QuotaFactory.UnboundedQuota
import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.EpochEndOffset._
import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest}
import org.apache.kafka.common.utils.SystemTime
import org.easymock.EasyMock._
import org.easymock.{Capture, CaptureType}

Loading…
Cancel
Save