mirror of https://github.com/apache/kafka.git
KAFKA-14991: KIP-937-Improve message timestamp validation (#14135)
This implementation introduces two new configurations `log.message.timestamp.before.max.ms` and `log.message.timestamp.after.max.ms` and deprecates `log.message.timestamp.difference.max.ms`. The default value for all these three configs is maintained to be Long.MAX_VALUE for backward compatibility but with the newly added configurations we can have a finer control when validating message timestamps that are in the past and the future compared to the broker's timestamp. To maintain backward compatibility if the default value of `log.message.timestamp.before.max.ms` is not changed, we are assuming users are still using the deprecated config `log.message.timestamp.difference.max.ms` and validation is done using its value. This ensures that existing customers who have customized the value of `log.message.timestamp.difference.max.ms` will continue to see no change in behavior. Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
This commit is contained in:
parent
09c074ad8a
commit
f91cb6b87b
|
@ -197,12 +197,35 @@ public class TopicConfig {
|
||||||
public static final String MESSAGE_TIMESTAMP_TYPE_DOC = "Define whether the timestamp in the message is " +
|
public static final String MESSAGE_TIMESTAMP_TYPE_DOC = "Define whether the timestamp in the message is " +
|
||||||
"message create time or log append time. The value should be either `CreateTime` or `LogAppendTime`";
|
"message create time or log append time. The value should be either `CreateTime` or `LogAppendTime`";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @deprecated since 3.6, removal planned in 4.0.
|
||||||
|
* Use message.timestamp.before.max.ms and message.timestamp.after.max.ms instead
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
public static final String MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG = "message.timestamp.difference.max.ms";
|
public static final String MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG = "message.timestamp.difference.max.ms";
|
||||||
public static final String MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC = "The maximum difference allowed between " +
|
|
||||||
|
/**
|
||||||
|
* @deprecated since 3.6, removal planned in 4.0.
|
||||||
|
* Use message.timestamp.before.max.ms and message.timestamp.after.max.ms instead
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
public static final String MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC = "[DEPRECATED] The maximum difference allowed between " +
|
||||||
"the timestamp when a broker receives a message and the timestamp specified in the message. If " +
|
"the timestamp when a broker receives a message and the timestamp specified in the message. If " +
|
||||||
"message.timestamp.type=CreateTime, a message will be rejected if the difference in timestamp " +
|
"message.timestamp.type=CreateTime, a message will be rejected if the difference in timestamp " +
|
||||||
"exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime.";
|
"exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime.";
|
||||||
|
|
||||||
|
public static final String MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG = "message.timestamp.before.max.ms";
|
||||||
|
public static final String MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC = "This configuration sets the allowable timestamp " +
|
||||||
|
"difference between the broker's timestamp and the message timestamp. The message timestamp can be earlier than " +
|
||||||
|
"or equal to the broker's timestamp, with the maximum allowable difference determined by the value set in this " +
|
||||||
|
"configuration. If message.timestamp.type=CreateTime, the message will be rejected if the difference in " +
|
||||||
|
"timestamps exceeds this specified threshold. This configuration is ignored if message.timestamp.type=LogAppendTime.";
|
||||||
|
public static final String MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG = "message.timestamp.after.max.ms";
|
||||||
|
public static final String MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC = "This configuration sets the allowable timestamp " +
|
||||||
|
"difference between the message timestamp and the broker's timestamp. The message timestamp can be later than " +
|
||||||
|
"or equal to the broker's timestamp, with the maximum allowable difference determined by the value set in this " +
|
||||||
|
"configuration. If message.timestamp.type=CreateTime, the message will be rejected if the difference in " +
|
||||||
|
"timestamps exceeds this specified threshold. This configuration is ignored if message.timestamp.type=LogAppendTime.";
|
||||||
public static final String MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = "message.downconversion.enable";
|
public static final String MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = "message.downconversion.enable";
|
||||||
public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "This configuration controls whether " +
|
public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "This configuration controls whether " +
|
||||||
"down-conversion of message formats is enabled to satisfy consume requests. When set to <code>false</code>, " +
|
"down-conversion of message formats is enabled to satisfy consume requests. When set to <code>false</code>, " +
|
||||||
|
|
|
@ -774,7 +774,8 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
config.compact,
|
config.compact,
|
||||||
config.recordVersion.value,
|
config.recordVersion.value,
|
||||||
config.messageTimestampType,
|
config.messageTimestampType,
|
||||||
config.messageTimestampDifferenceMaxMs,
|
config.messageTimestampBeforeMaxMs,
|
||||||
|
config.messageTimestampAfterMaxMs,
|
||||||
leaderEpoch,
|
leaderEpoch,
|
||||||
origin,
|
origin,
|
||||||
interBrokerProtocolVersion
|
interBrokerProtocolVersion
|
||||||
|
|
|
@ -465,7 +465,14 @@ object KafkaConfig {
|
||||||
val LogMessageFormatVersionProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)
|
val LogMessageFormatVersionProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)
|
||||||
|
|
||||||
val LogMessageTimestampTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG)
|
val LogMessageTimestampTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG)
|
||||||
|
|
||||||
|
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
|
||||||
|
@deprecated("3.6")
|
||||||
val LogMessageTimestampDifferenceMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG)
|
val LogMessageTimestampDifferenceMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG)
|
||||||
|
|
||||||
|
val LogMessageTimestampBeforeMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)
|
||||||
|
val LogMessageTimestampAfterMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)
|
||||||
|
|
||||||
val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir"
|
val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir"
|
||||||
val AutoCreateTopicsEnableProp = "auto.create.topics.enable"
|
val AutoCreateTopicsEnableProp = "auto.create.topics.enable"
|
||||||
val MinInSyncReplicasProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG)
|
val MinInSyncReplicasProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG)
|
||||||
|
@ -899,10 +906,23 @@ object KafkaConfig {
|
||||||
val LogMessageTimestampTypeDoc = "Define whether the timestamp in the message is message create time or log append time. The value should be either " +
|
val LogMessageTimestampTypeDoc = "Define whether the timestamp in the message is message create time or log append time. The value should be either " +
|
||||||
"`CreateTime` or `LogAppendTime`."
|
"`CreateTime` or `LogAppendTime`."
|
||||||
|
|
||||||
val LogMessageTimestampDifferenceMaxMsDoc = "The maximum difference allowed between the timestamp when a broker receives " +
|
val LogMessageTimestampDifferenceMaxMsDoc = "[DEPRECATED] The maximum difference allowed between the timestamp when a broker receives " +
|
||||||
"a message and the timestamp specified in the message. If log.message.timestamp.type=CreateTime, a message will be rejected " +
|
"a message and the timestamp specified in the message. If log.message.timestamp.type=CreateTime, a message will be rejected " +
|
||||||
"if the difference in timestamp exceeds this threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." +
|
"if the difference in timestamp exceeds this threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." +
|
||||||
"The maximum timestamp difference allowed should be no greater than log.retention.ms to avoid unnecessarily frequent log rolling."
|
"The maximum timestamp difference allowed should be no greater than log.retention.ms to avoid unnecessarily frequent log rolling."
|
||||||
|
|
||||||
|
val LogMessageTimestampBeforeMaxMsDoc = "This configuration sets the allowable timestamp difference between the " +
|
||||||
|
"broker's timestamp and the message timestamp. The message timestamp can be earlier than or equal to the broker's " +
|
||||||
|
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
|
||||||
|
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
|
||||||
|
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime.";
|
||||||
|
|
||||||
|
val LogMessageTimestampAfterMaxMsDoc = "This configuration sets the allowable timestamp difference between the " +
|
||||||
|
"message timestamp and the broker's timestamp. The message timestamp can be later than or equal to the broker's " +
|
||||||
|
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
|
||||||
|
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
|
||||||
|
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime.";
|
||||||
|
|
||||||
val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown"
|
val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown"
|
||||||
val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server."
|
val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server."
|
||||||
val MinInSyncReplicasDoc = "When a producer sets acks to \"all\" (or \"-1\"), " +
|
val MinInSyncReplicasDoc = "When a producer sets acks to \"all\" (or \"-1\"), " +
|
||||||
|
@ -1284,6 +1304,8 @@ object KafkaConfig {
|
||||||
.define(LogMessageFormatVersionProp, STRING, LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM, LogMessageFormatVersionDoc)
|
.define(LogMessageFormatVersionProp, STRING, LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM, LogMessageFormatVersionDoc)
|
||||||
.define(LogMessageTimestampTypeProp, STRING, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_TYPE, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc)
|
.define(LogMessageTimestampTypeProp, STRING, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_TYPE, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc)
|
||||||
.define(LogMessageTimestampDifferenceMaxMsProp, LONG, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc)
|
.define(LogMessageTimestampDifferenceMaxMsProp, LONG, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc)
|
||||||
|
.define(LogMessageTimestampBeforeMaxMsProp, LONG, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_BEFORE_MAX_MS, atLeast(0), MEDIUM, LogMessageTimestampBeforeMaxMsDoc)
|
||||||
|
.define(LogMessageTimestampAfterMaxMsProp, LONG, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_AFTER_MAX_MS, atLeast(0), MEDIUM, LogMessageTimestampAfterMaxMsDoc)
|
||||||
.define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc)
|
.define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc)
|
||||||
.define(AlterConfigPolicyClassNameProp, CLASS, null, LOW, AlterConfigPolicyClassNameDoc)
|
.define(AlterConfigPolicyClassNameProp, CLASS, null, LOW, AlterConfigPolicyClassNameDoc)
|
||||||
.define(LogMessageDownConversionEnableProp, BOOLEAN, LogConfig.DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, LogMessageDownConversionEnableDoc)
|
.define(LogMessageDownConversionEnableProp, BOOLEAN, LogConfig.DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, LogMessageDownConversionEnableDoc)
|
||||||
|
@ -1862,7 +1884,37 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
||||||
else MetadataVersion.fromVersionString(logMessageFormatVersionString)
|
else MetadataVersion.fromVersionString(logMessageFormatVersionString)
|
||||||
|
|
||||||
def logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp))
|
def logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp))
|
||||||
|
|
||||||
|
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
|
||||||
|
@deprecated("3.6")
|
||||||
def logMessageTimestampDifferenceMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)
|
def logMessageTimestampDifferenceMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)
|
||||||
|
|
||||||
|
// In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
|
||||||
|
// we are using its value if logMessageTimestampBeforeMaxMs default value hasn't changed.
|
||||||
|
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
|
||||||
|
@nowarn("cat=deprecation")
|
||||||
|
def logMessageTimestampBeforeMaxMs: Long = {
|
||||||
|
val messageTimestampBeforeMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampBeforeMaxMsProp)
|
||||||
|
if (messageTimestampBeforeMaxMs != LogConfig.DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS) {
|
||||||
|
messageTimestampBeforeMaxMs
|
||||||
|
} else {
|
||||||
|
logMessageTimestampDifferenceMaxMs
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
|
||||||
|
// we are using its value if logMessageTimestampAfterMaxMs default value hasn't changed.
|
||||||
|
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
|
||||||
|
@nowarn("cat=deprecation")
|
||||||
|
def logMessageTimestampAfterMaxMs: Long = {
|
||||||
|
val messageTimestampAfterMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampAfterMaxMsProp)
|
||||||
|
if (messageTimestampAfterMaxMs != Long.MaxValue) {
|
||||||
|
messageTimestampAfterMaxMs
|
||||||
|
} else {
|
||||||
|
logMessageTimestampDifferenceMaxMs
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
def logMessageDownConversionEnable: Boolean = getBoolean(KafkaConfig.LogMessageDownConversionEnableProp)
|
def logMessageDownConversionEnable: Boolean = getBoolean(KafkaConfig.LogMessageDownConversionEnableProp)
|
||||||
|
|
||||||
/** ********* Replication configuration ***********/
|
/** ********* Replication configuration ***********/
|
||||||
|
@ -2450,6 +2502,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
||||||
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version)
|
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version)
|
||||||
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, logMessageTimestampType.name)
|
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, logMessageTimestampType.name)
|
||||||
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, logMessageTimestampDifferenceMaxMs: java.lang.Long)
|
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, logMessageTimestampDifferenceMaxMs: java.lang.Long)
|
||||||
|
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, logMessageTimestampBeforeMaxMs: java.lang.Long)
|
||||||
|
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, logMessageTimestampAfterMaxMs: java.lang.Long)
|
||||||
logProps.put(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, logMessageDownConversionEnable: java.lang.Boolean)
|
logProps.put(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, logMessageDownConversionEnable: java.lang.Boolean)
|
||||||
logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, logLocalRetentionMs)
|
logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, logLocalRetentionMs)
|
||||||
logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, logLocalRetentionBytes)
|
logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, logLocalRetentionBytes)
|
||||||
|
|
|
@ -29,9 +29,10 @@ import org.apache.kafka.storage.internals.log.LogConfig
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.Timeout
|
import org.junit.jupiter.api.Timeout
|
||||||
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.{Arguments, MethodSource, ValueSource}
|
||||||
|
|
||||||
import java.nio.charset.StandardCharsets
|
import java.nio.charset.StandardCharsets
|
||||||
|
import scala.annotation.nowarn
|
||||||
|
|
||||||
|
|
||||||
class PlaintextProducerSendTest extends BaseProducerSendTest {
|
class PlaintextProducerSendTest extends BaseProducerSendTest {
|
||||||
|
@ -121,16 +122,18 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
|
||||||
@ValueSource(strings = Array("zk", "kraft"))
|
@MethodSource(Array("quorumAndTimestampConfigProvider"))
|
||||||
def testSendWithInvalidCreateTime(quorum: String): Unit = {
|
def testSendWithInvalidBeforeAndAfterTimestamp(quorum: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
|
||||||
val topicProps = new Properties()
|
val topicProps = new Properties()
|
||||||
topicProps.setProperty(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000")
|
// set the TopicConfig for timestamp validation to have 1 minute threshold. Note that recordTimestamp has 5 minutes diff
|
||||||
|
val oneMinuteInMs: Long = 1 * 60 * 60 * 1000L
|
||||||
|
topicProps.setProperty(messageTimeStampConfig, oneMinuteInMs.toString)
|
||||||
TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps)
|
TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps)
|
||||||
|
|
||||||
val producer = createProducer()
|
val producer = createProducer()
|
||||||
try {
|
try {
|
||||||
val e = assertThrows(classOf[ExecutionException],
|
val e = assertThrows(classOf[ExecutionException],
|
||||||
() => producer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get()).getCause
|
() => producer.send(new ProducerRecord(topic, 0, recordTimestamp, "key".getBytes, "value".getBytes)).get()).getCause
|
||||||
assertTrue(e.isInstanceOf[InvalidTimestampException])
|
assertTrue(e.isInstanceOf[InvalidTimestampException])
|
||||||
} finally {
|
} finally {
|
||||||
producer.close()
|
producer.close()
|
||||||
|
@ -140,13 +143,54 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
|
||||||
val compressedProducer = createProducer(compressionType = "gzip")
|
val compressedProducer = createProducer(compressionType = "gzip")
|
||||||
try {
|
try {
|
||||||
val e = assertThrows(classOf[ExecutionException],
|
val e = assertThrows(classOf[ExecutionException],
|
||||||
() => compressedProducer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get()).getCause
|
() => compressedProducer.send(new ProducerRecord(topic, 0, recordTimestamp, "key".getBytes, "value".getBytes)).get()).getCause
|
||||||
assertTrue(e.isInstanceOf[InvalidTimestampException])
|
assertTrue(e.isInstanceOf[InvalidTimestampException])
|
||||||
} finally {
|
} finally {
|
||||||
compressedProducer.close()
|
compressedProducer.close()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource(Array("quorumAndTimestampConfigProvider"))
|
||||||
|
def testValidBeforeAndAfterTimestampsAtThreshold(quorum: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
|
||||||
|
val topicProps = new Properties()
|
||||||
|
|
||||||
|
// set the TopicConfig for timestamp validation to be the same as the record timestamp
|
||||||
|
topicProps.setProperty(messageTimeStampConfig, recordTimestamp.toString)
|
||||||
|
TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps)
|
||||||
|
|
||||||
|
val producer = createProducer()
|
||||||
|
|
||||||
|
assertDoesNotThrow(() => producer.send(new ProducerRecord(topic, 0, recordTimestamp, "key".getBytes, "value".getBytes)))
|
||||||
|
producer.close()
|
||||||
|
|
||||||
|
// Test compressed messages.
|
||||||
|
val compressedProducer = createProducer(compressionType = "gzip")
|
||||||
|
assertDoesNotThrow(() => compressedProducer.send(new ProducerRecord(topic, 0, recordTimestamp, "key".getBytes, "value".getBytes)))
|
||||||
|
compressedProducer.close()
|
||||||
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource(Array("quorumAndTimestampConfigProvider"))
|
||||||
|
def testValidBeforeAndAfterTimestampsWithinThreshold(quorum: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
|
||||||
|
val topicProps = new Properties()
|
||||||
|
|
||||||
|
// set the TopicConfig for timestamp validation to have 10 minute threshold. Note that recordTimestamp has 5 minutes diff
|
||||||
|
val tenMinutesInMs: Long = 10 * 60 * 60 * 1000L
|
||||||
|
topicProps.setProperty(messageTimeStampConfig, tenMinutesInMs.toString)
|
||||||
|
TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps)
|
||||||
|
|
||||||
|
val producer = createProducer()
|
||||||
|
|
||||||
|
assertDoesNotThrow(() => producer.send(new ProducerRecord(topic, 0, recordTimestamp, "key".getBytes, "value".getBytes)))
|
||||||
|
producer.close()
|
||||||
|
|
||||||
|
// Test compressed messages.
|
||||||
|
val compressedProducer = createProducer(compressionType = "gzip")
|
||||||
|
assertDoesNotThrow(() => compressedProducer.send(new ProducerRecord(topic, 0, recordTimestamp, "key".getBytes, "value".getBytes)))
|
||||||
|
compressedProducer.close()
|
||||||
|
}
|
||||||
|
|
||||||
// Test that producer with max.block.ms=0 can be used to send in non-blocking mode
|
// Test that producer with max.block.ms=0 can be used to send in non-blocking mode
|
||||||
// where requests are failed immediately without blocking if metadata is not available
|
// where requests are failed immediately without blocking if metadata is not available
|
||||||
// or buffer is full.
|
// or buffer is full.
|
||||||
|
@ -228,3 +272,21 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
object PlaintextProducerSendTest {
|
||||||
|
|
||||||
|
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
|
||||||
|
@nowarn("cat=deprecation")
|
||||||
|
def quorumAndTimestampConfigProvider: java.util.stream.Stream[Arguments] = {
|
||||||
|
val now: Long = System.currentTimeMillis()
|
||||||
|
val fiveMinutesInMs: Long = 5 * 60 * 60 * 1000L
|
||||||
|
java.util.stream.Stream.of[Arguments](
|
||||||
|
Arguments.of("zk", TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
|
||||||
|
Arguments.of("zk", TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
|
||||||
|
Arguments.of("zk", TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.box(now + fiveMinutesInMs)),
|
||||||
|
Arguments.of("kraft", TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
|
||||||
|
Arguments.of("kraft", TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
|
||||||
|
Arguments.of("kraft", TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.box(now + fiveMinutesInMs))
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
|
@ -618,6 +618,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Disabled // TODO: To be re-enabled once we can make it less flaky: KAFKA-6527
|
@Disabled // TODO: To be re-enabled once we can make it less flaky: KAFKA-6527
|
||||||
|
@nowarn("cat=deprecation") // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
|
||||||
def testDefaultTopicConfig(): Unit = {
|
def testDefaultTopicConfig(): Unit = {
|
||||||
val (producerThread, consumerThread) = startProduceConsume(retries = 0)
|
val (producerThread, consumerThread) = startProduceConsume(retries = 0)
|
||||||
|
|
||||||
|
@ -643,6 +644,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
|
||||||
props.put(KafkaConfig.LogPreAllocateProp, true.toString)
|
props.put(KafkaConfig.LogPreAllocateProp, true.toString)
|
||||||
props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.toString)
|
props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.toString)
|
||||||
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000")
|
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000")
|
||||||
|
props.put(KafkaConfig.LogMessageTimestampBeforeMaxMsProp, "1000")
|
||||||
|
props.put(KafkaConfig.LogMessageTimestampAfterMaxMsProp, "1000")
|
||||||
props.put(KafkaConfig.LogMessageDownConversionEnableProp, "false")
|
props.put(KafkaConfig.LogMessageDownConversionEnableProp, "false")
|
||||||
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogSegmentBytesProp, "4000"))
|
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogSegmentBytesProp, "4000"))
|
||||||
|
|
||||||
|
@ -681,11 +684,15 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
|
||||||
props.clear()
|
props.clear()
|
||||||
props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.CREATE_TIME.toString)
|
props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.CREATE_TIME.toString)
|
||||||
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000")
|
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000")
|
||||||
|
props.put(KafkaConfig.LogMessageTimestampBeforeMaxMsProp, "1000")
|
||||||
|
props.put(KafkaConfig.LogMessageTimestampAfterMaxMsProp, "1000")
|
||||||
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogMessageTimestampTypeProp, TimestampType.CREATE_TIME.toString))
|
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogMessageTimestampTypeProp, TimestampType.CREATE_TIME.toString))
|
||||||
consumerThread.waitForMatchingRecords(record => record.timestampType == TimestampType.CREATE_TIME)
|
consumerThread.waitForMatchingRecords(record => record.timestampType == TimestampType.CREATE_TIME)
|
||||||
// Verify that invalid configs are not applied
|
// Verify that invalid configs are not applied
|
||||||
val invalidProps = Map(
|
val invalidProps = Map(
|
||||||
KafkaConfig.LogMessageTimestampDifferenceMaxMsProp -> "abc", // Invalid type
|
KafkaConfig.LogMessageTimestampDifferenceMaxMsProp -> "abc", // Invalid type
|
||||||
|
KafkaConfig.LogMessageTimestampBeforeMaxMsProp -> "abc", // Invalid type
|
||||||
|
KafkaConfig.LogMessageTimestampAfterMaxMsProp -> "abc", // Invalid type
|
||||||
KafkaConfig.LogMessageTimestampTypeProp -> "invalid", // Invalid value
|
KafkaConfig.LogMessageTimestampTypeProp -> "invalid", // Invalid value
|
||||||
KafkaConfig.LogRollTimeMillisProp -> "0" // Fails KafkaConfig validation
|
KafkaConfig.LogRollTimeMillisProp -> "0" // Fails KafkaConfig validation
|
||||||
)
|
)
|
||||||
|
|
|
@ -73,7 +73,6 @@ abstract class AbstractLogCleanerIntegrationTest {
|
||||||
props.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, deleteDelay: java.lang.Integer)
|
props.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, deleteDelay: java.lang.Integer)
|
||||||
props.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
|
props.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
|
||||||
props.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, minCleanableDirtyRatio: java.lang.Float)
|
props.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, minCleanableDirtyRatio: java.lang.Float)
|
||||||
props.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.MaxValue.toString)
|
|
||||||
props.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, minCompactionLagMs: java.lang.Long)
|
props.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, minCompactionLagMs: java.lang.Long)
|
||||||
props.put(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, maxCompactionLagMs: java.lang.Long)
|
props.put(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, maxCompactionLagMs: java.lang.Long)
|
||||||
props ++= propertyOverrides
|
props ++= propertyOverrides
|
||||||
|
|
|
@ -54,7 +54,6 @@ class LogCleanerTest {
|
||||||
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
|
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
|
||||||
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1024: java.lang.Integer)
|
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1024: java.lang.Integer)
|
||||||
logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
|
logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
|
||||||
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.MaxValue.toString)
|
|
||||||
val logConfig = new LogConfig(logProps)
|
val logConfig = new LogConfig(logProps)
|
||||||
val time = new MockTime()
|
val time = new MockTime()
|
||||||
val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time)
|
val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time)
|
||||||
|
|
|
@ -397,4 +397,22 @@ class LogConfigTest {
|
||||||
LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)
|
LogConfig.validateBrokerLogConfigValues(kafkaConfig.extractLogConfigMap, kafkaConfig.isRemoteLogStorageSystemEnabled)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/* Verify that when the deprecated config LogMessageTimestampDifferenceMaxMsProp has non default value the new configs
|
||||||
|
* LogMessageTimestampBeforeMaxMsProp and LogMessageTimestampAfterMaxMsProp are not changed from the default we are using
|
||||||
|
* the deprecated config for backward compatibility.
|
||||||
|
* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details */
|
||||||
|
@nowarn("cat=deprecation")
|
||||||
|
@Test
|
||||||
|
def testTimestampBeforeMaxMsUsesDeprecatedConfig(): Unit = {
|
||||||
|
val oneDayInMillis = 24 * 60 * 60 * 1000L
|
||||||
|
val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
|
||||||
|
kafkaProps.put(KafkaConfig.LogMessageTimestampBeforeMaxMsProp, Long.MaxValue.toString)
|
||||||
|
kafkaProps.put(KafkaConfig.LogMessageTimestampAfterMaxMsProp, Long.MaxValue.toString)
|
||||||
|
kafkaProps.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, oneDayInMillis.toString)
|
||||||
|
|
||||||
|
val logProps = KafkaConfig.fromProps(kafkaProps).extractLogConfigMap
|
||||||
|
assertEquals(oneDayInMillis, logProps.get(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG))
|
||||||
|
assertEquals(oneDayInMillis, logProps.get(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG))
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -55,7 +55,6 @@ class LogManagerTest {
|
||||||
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
|
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
|
||||||
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer)
|
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer)
|
||||||
logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer)
|
logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer)
|
||||||
logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.MaxValue.toString)
|
|
||||||
val logConfig = new LogConfig(logProps)
|
val logConfig = new LogConfig(logProps)
|
||||||
var logDir: File = _
|
var logDir: File = _
|
||||||
var logManager: LogManager = _
|
var logManager: LogManager = _
|
||||||
|
|
|
@ -123,6 +123,7 @@ class LogValidatorTest {
|
||||||
magic,
|
magic,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PRODUCER_EPOCH,
|
RecordBatch.NO_PRODUCER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.IBP_2_3_IV1
|
MetadataVersion.IBP_2_3_IV1
|
||||||
|
@ -154,6 +155,7 @@ class LogValidatorTest {
|
||||||
magic,
|
magic,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -198,6 +200,7 @@ class LogValidatorTest {
|
||||||
targetMagic,
|
targetMagic,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -249,6 +252,7 @@ class LogValidatorTest {
|
||||||
magic,
|
magic,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -314,6 +318,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -361,6 +366,7 @@ class LogValidatorTest {
|
||||||
magic,
|
magic,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
partitionLeaderEpoch,
|
partitionLeaderEpoch,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -438,6 +444,7 @@ class LogValidatorTest {
|
||||||
magic,
|
magic,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
partitionLeaderEpoch,
|
partitionLeaderEpoch,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -496,6 +503,7 @@ class LogValidatorTest {
|
||||||
toMagic,
|
toMagic,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -543,6 +551,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest,
|
MetadataVersion.latest,
|
||||||
|
@ -602,6 +611,7 @@ class LogValidatorTest {
|
||||||
magic,
|
magic,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
partitionLeaderEpoch,
|
partitionLeaderEpoch,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -657,6 +667,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -682,6 +693,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -707,6 +719,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -732,6 +745,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -756,6 +770,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V0,
|
RecordBatch.MAGIC_VALUE_V0,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest,
|
MetadataVersion.latest,
|
||||||
|
@ -778,6 +793,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V0,
|
RecordBatch.MAGIC_VALUE_V0,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -801,6 +817,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -825,6 +842,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -850,6 +868,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -875,6 +894,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -898,6 +918,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -923,6 +944,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -948,6 +970,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -973,6 +996,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -998,6 +1022,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1020,6 +1045,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.COORDINATOR,
|
AppendOrigin.COORDINATOR,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1047,6 +1073,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V0,
|
RecordBatch.MAGIC_VALUE_V0,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1070,6 +1097,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V0,
|
RecordBatch.MAGIC_VALUE_V0,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1092,6 +1120,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1114,6 +1143,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1137,6 +1167,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1160,6 +1191,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1185,6 +1217,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1210,6 +1243,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1233,6 +1267,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V0,
|
RecordBatch.MAGIC_VALUE_V0,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1256,6 +1291,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V0,
|
RecordBatch.MAGIC_VALUE_V0,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1277,6 +1313,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V0,
|
RecordBatch.MAGIC_VALUE_V0,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1305,6 +1342,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V2,
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
TimestampType.LOG_APPEND_TIME,
|
TimestampType.LOG_APPEND_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.IBP_2_0_IV1
|
MetadataVersion.IBP_2_0_IV1
|
||||||
|
@ -1339,6 +1377,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.MAGIC_VALUE_V1,
|
RecordBatch.MAGIC_VALUE_V1,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
1000L,
|
1000L,
|
||||||
|
1000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
@ -1398,6 +1437,73 @@ class LogValidatorTest {
|
||||||
assertEquals(6, e.recordErrors.size)
|
assertEquals(6, e.recordErrors.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testRecordWithPastTimestampIsRejected(): Unit = {
|
||||||
|
val timestampBeforeMaxConfig = 24 * 60 * 60 * 1000L //24 hrs
|
||||||
|
val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val fiveMinutesBeforeThreshold = now - timestampBeforeMaxConfig - (5 * 60 * 1000L)
|
||||||
|
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesBeforeThreshold,
|
||||||
|
codec = CompressionType.GZIP)
|
||||||
|
val e = assertThrows(classOf[RecordValidationException],
|
||||||
|
() => new LogValidator(
|
||||||
|
records,
|
||||||
|
topicPartition,
|
||||||
|
time,
|
||||||
|
CompressionType.GZIP,
|
||||||
|
CompressionType.GZIP,
|
||||||
|
false,
|
||||||
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
|
TimestampType.CREATE_TIME,
|
||||||
|
timestampBeforeMaxConfig,
|
||||||
|
timestampAfterMaxConfig,
|
||||||
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
|
AppendOrigin.CLIENT,
|
||||||
|
MetadataVersion.latest
|
||||||
|
).validateMessagesAndAssignOffsets(
|
||||||
|
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
assertTrue(e.invalidException.isInstanceOf[InvalidTimestampException])
|
||||||
|
assertFalse(e.recordErrors.isEmpty)
|
||||||
|
assertEquals(e.recordErrors.size, 3)
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testRecordWithFutureTimestampIsRejected(): Unit = {
|
||||||
|
val timestampBeforeMaxConfig = 24 * 60 * 60 * 1000L //24 hrs
|
||||||
|
val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val fiveMinutesAfterThreshold = now + timestampAfterMaxConfig + (5 * 60 * 1000L)
|
||||||
|
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesAfterThreshold,
|
||||||
|
codec = CompressionType.GZIP)
|
||||||
|
val e = assertThrows(classOf[RecordValidationException],
|
||||||
|
() => new LogValidator(
|
||||||
|
records,
|
||||||
|
topicPartition,
|
||||||
|
time,
|
||||||
|
CompressionType.GZIP,
|
||||||
|
CompressionType.GZIP,
|
||||||
|
false,
|
||||||
|
RecordBatch.MAGIC_VALUE_V2,
|
||||||
|
TimestampType.CREATE_TIME,
|
||||||
|
timestampBeforeMaxConfig,
|
||||||
|
timestampAfterMaxConfig,
|
||||||
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
|
AppendOrigin.CLIENT,
|
||||||
|
MetadataVersion.latest
|
||||||
|
).validateMessagesAndAssignOffsets(
|
||||||
|
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
assertTrue(e.invalidException.isInstanceOf[InvalidTimestampException])
|
||||||
|
assertFalse(e.recordErrors.isEmpty)
|
||||||
|
assertEquals(e.recordErrors.size, 3)
|
||||||
|
}
|
||||||
|
|
||||||
private def testBatchWithoutRecordsNotAllowed(sourceCompression: CompressionType, targetCompression: CompressionType): Unit = {
|
private def testBatchWithoutRecordsNotAllowed(sourceCompression: CompressionType, targetCompression: CompressionType): Unit = {
|
||||||
val offset = 1234567
|
val offset = 1234567
|
||||||
val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
|
val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
|
||||||
|
@ -1417,6 +1523,7 @@ class LogValidatorTest {
|
||||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||||
TimestampType.CREATE_TIME,
|
TimestampType.CREATE_TIME,
|
||||||
5000L,
|
5000L,
|
||||||
|
5000L,
|
||||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||||
AppendOrigin.CLIENT,
|
AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest
|
MetadataVersion.latest
|
||||||
|
|
|
@ -761,6 +761,7 @@ class KafkaConfigTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@nowarn("cat=deprecation") // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
|
||||||
def testFromPropsInvalid(): Unit = {
|
def testFromPropsInvalid(): Unit = {
|
||||||
def baseProperties: Properties = {
|
def baseProperties: Properties = {
|
||||||
val validRequiredProperties = new Properties()
|
val validRequiredProperties = new Properties()
|
||||||
|
@ -856,6 +857,8 @@ class KafkaConfigTest {
|
||||||
case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||||
case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||||
case KafkaConfig.LogMessageTimestampDifferenceMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
case KafkaConfig.LogMessageTimestampDifferenceMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||||
|
case KafkaConfig.LogMessageTimestampBeforeMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||||
|
case KafkaConfig.LogMessageTimestampAfterMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||||
case KafkaConfig.LogFlushStartOffsetCheckpointIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
case KafkaConfig.LogFlushStartOffsetCheckpointIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
|
||||||
case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
|
case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
|
||||||
case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0")
|
case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0")
|
||||||
|
@ -1094,6 +1097,10 @@ class KafkaConfigTest {
|
||||||
assertDynamic(kafkaConfigProp, false, () => config.logMessageDownConversionEnable)
|
assertDynamic(kafkaConfigProp, false, () => config.logMessageDownConversionEnable)
|
||||||
case TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG =>
|
case TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG =>
|
||||||
assertDynamic(kafkaConfigProp, 10009, () => config.logMessageTimestampDifferenceMaxMs)
|
assertDynamic(kafkaConfigProp, 10009, () => config.logMessageTimestampDifferenceMaxMs)
|
||||||
|
case TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG =>
|
||||||
|
assertDynamic(kafkaConfigProp, 10015L, () => config.logMessageTimestampBeforeMaxMs)
|
||||||
|
case TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG =>
|
||||||
|
assertDynamic(kafkaConfigProp, 10016L, () => config.logMessageTimestampAfterMaxMs)
|
||||||
case TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG =>
|
case TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG =>
|
||||||
assertDynamic(kafkaConfigProp, "LogAppendTime", () => config.logMessageTimestampType.name)
|
assertDynamic(kafkaConfigProp, "LogAppendTime", () => config.logMessageTimestampType.name)
|
||||||
case TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG =>
|
case TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG =>
|
||||||
|
|
|
@ -30,7 +30,10 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||||
import org.apache.kafka.server.record.BrokerCompressionType
|
import org.apache.kafka.server.record.BrokerCompressionType
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.Test
|
import org.junit.jupiter.api.Test
|
||||||
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
|
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
|
||||||
|
|
||||||
|
import scala.annotation.nowarn
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -78,12 +81,13 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
new SimpleRecord(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
|
new SimpleRecord(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest
|
||||||
def testProduceWithInvalidTimestamp(): Unit = {
|
@MethodSource(Array("timestampConfigProvider"))
|
||||||
|
def testProduceWithInvalidTimestamp(messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val partition = 0
|
val partition = 0
|
||||||
val topicConfig = new Properties
|
val topicConfig = new Properties
|
||||||
topicConfig.setProperty(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000")
|
topicConfig.setProperty(messageTimeStampConfig, "1000")
|
||||||
val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig)
|
val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig)
|
||||||
val leader = partitionToLeader(partition)
|
val leader = partitionToLeader(partition)
|
||||||
|
|
||||||
|
@ -96,7 +100,7 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
builder.build()
|
builder.build()
|
||||||
}
|
}
|
||||||
|
|
||||||
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, System.currentTimeMillis() - 1001L, CompressionType.GZIP)
|
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, recordTimestamp, CompressionType.GZIP)
|
||||||
val topicPartition = new TopicPartition("topic", partition)
|
val topicPartition = new TopicPartition("topic", partition)
|
||||||
val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||||
|
@ -250,3 +254,16 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
object ProduceRequestTest {
|
||||||
|
|
||||||
|
@nowarn("cat=deprecation") // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
|
||||||
|
def timestampConfigProvider: java.util.stream.Stream[Arguments] = {
|
||||||
|
val fiveMinutesInMs: Long = 5 * 60 * 60 * 1000L
|
||||||
|
java.util.stream.Stream.of[Arguments](
|
||||||
|
Arguments.of(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.box(System.currentTimeMillis() - fiveMinutesInMs)),
|
||||||
|
Arguments.of(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.box(System.currentTimeMillis() - fiveMinutesInMs)),
|
||||||
|
Arguments.of(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.box(System.currentTimeMillis() + fiveMinutesInMs))
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -369,7 +369,6 @@ object TestUtils extends Logging {
|
||||||
props.put(KafkaConfig.LogDeleteDelayMsProp, "1000")
|
props.put(KafkaConfig.LogDeleteDelayMsProp, "1000")
|
||||||
props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100")
|
props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100")
|
||||||
props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "2097152")
|
props.put(KafkaConfig.LogCleanerDedupeBufferSizeProp, "2097152")
|
||||||
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString)
|
|
||||||
props.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
|
props.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
|
||||||
if (!props.containsKey(KafkaConfig.OffsetsTopicPartitionsProp))
|
if (!props.containsKey(KafkaConfig.OffsetsTopicPartitionsProp))
|
||||||
props.put(KafkaConfig.OffsetsTopicPartitionsProp, "5")
|
props.put(KafkaConfig.OffsetsTopicPartitionsProp, "5")
|
||||||
|
|
|
@ -37,6 +37,9 @@
|
||||||
For more detailed information please refer to the Compatibility, Deprecation, and Migration Plan section in
|
For more detailed information please refer to the Compatibility, Deprecation, and Migration Plan section in
|
||||||
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-902%3A+Upgrade+Zookeeper+to+3.8.1">KIP-902</a>.
|
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-902%3A+Upgrade+Zookeeper+to+3.8.1">KIP-902</a>.
|
||||||
</li>
|
</li>
|
||||||
|
<li>The configuration <code>log.message.timestamp.difference.max.ms</code> is deprecated.
|
||||||
|
Two new configurations, <code>log.message.timestamp.before.max.ms</code> and <code>log.message.timestamp.after.max.ms</code>, have been added.
|
||||||
|
For more detailed information, please refer to <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-937%3A+Improve+Message+Timestamp+Validation">KIP-937</a>.
|
||||||
<li>
|
<li>
|
||||||
Kafka Streams has introduced a new task assignor, <code>RackAwareTaskAssignor</code>, for computing task assignments which can minimize
|
Kafka Streams has introduced a new task assignor, <code>RackAwareTaskAssignor</code>, for computing task assignments which can minimize
|
||||||
cross rack traffic under certain conditions. It works with existing <code>StickyTaskAssignor</code> and <code>HighAvailabilityTaskAssignor</code>.
|
cross rack traffic under certain conditions. It works with existing <code>StickyTaskAssignor</code> and <code>HighAvailabilityTaskAssignor</code>.
|
||||||
|
|
|
@ -53,7 +53,7 @@ public class CompressedRecordBatchValidationBenchmark extends BaseRecordBatchBen
|
||||||
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
||||||
new LogValidator(records, new TopicPartition("a", 0),
|
new LogValidator(records, new TopicPartition("a", 0),
|
||||||
Time.SYSTEM, compressionType, compressionType, false, messageVersion,
|
Time.SYSTEM, compressionType, compressionType, false, messageVersion,
|
||||||
TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
|
TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest()
|
MetadataVersion.latest()
|
||||||
).validateMessagesAndAssignOffsetsCompressed(PrimitiveRef.ofLong(startingOffset),
|
).validateMessagesAndAssignOffsetsCompressed(PrimitiveRef.ofLong(startingOffset),
|
||||||
validatorMetricsRecorder, requestLocal.bufferSupplier());
|
validatorMetricsRecorder, requestLocal.bufferSupplier());
|
||||||
|
|
|
@ -49,7 +49,7 @@ public class UncompressedRecordBatchValidationBenchmark extends BaseRecordBatchB
|
||||||
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
|
||||||
new LogValidator(records, new TopicPartition("a", 0),
|
new LogValidator(records, new TopicPartition("a", 0),
|
||||||
Time.SYSTEM, CompressionType.NONE, CompressionType.NONE, false,
|
Time.SYSTEM, CompressionType.NONE, CompressionType.NONE, false,
|
||||||
messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
|
messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
|
||||||
MetadataVersion.latest()
|
MetadataVersion.latest()
|
||||||
).assignOffsetsNonCompressed(PrimitiveRef.ofLong(startingOffset), validatorMetricsRecorder);
|
).assignOffsetsNonCompressed(PrimitiveRef.ofLong(startingOffset), validatorMetricsRecorder);
|
||||||
}
|
}
|
||||||
|
|
|
@ -83,6 +83,8 @@ public final class ServerTopicConfigSynonyms {
|
||||||
sameNameWithLogPrefix(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG),
|
sameNameWithLogPrefix(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG),
|
||||||
sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG),
|
sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG),
|
||||||
sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG),
|
sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG),
|
||||||
|
sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG),
|
||||||
|
sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG),
|
||||||
sameNameWithLogPrefix(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG),
|
sameNameWithLogPrefix(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG),
|
||||||
sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG),
|
sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG),
|
||||||
sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)
|
sameNameWithLogPrefix(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)
|
||||||
|
|
|
@ -175,7 +175,12 @@ public class LogConfig extends AbstractConfig {
|
||||||
public static final String DEFAULT_COMPRESSION_TYPE = BrokerCompressionType.PRODUCER.name;
|
public static final String DEFAULT_COMPRESSION_TYPE = BrokerCompressionType.PRODUCER.name;
|
||||||
public static final boolean DEFAULT_PREALLOCATE = false;
|
public static final boolean DEFAULT_PREALLOCATE = false;
|
||||||
public static final String DEFAULT_MESSAGE_TIMESTAMP_TYPE = "CreateTime";
|
public static final String DEFAULT_MESSAGE_TIMESTAMP_TYPE = "CreateTime";
|
||||||
|
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
|
||||||
|
@Deprecated
|
||||||
public static final long DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS = Long.MAX_VALUE;
|
public static final long DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS = Long.MAX_VALUE;
|
||||||
|
|
||||||
|
public static final long DEFAULT_MESSAGE_TIMESTAMP_BEFORE_MAX_MS = Long.MAX_VALUE;
|
||||||
|
public static final long DEFAULT_MESSAGE_TIMESTAMP_AFTER_MAX_MS = Long.MAX_VALUE;
|
||||||
public static final boolean DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE = true;
|
public static final boolean DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE = true;
|
||||||
|
|
||||||
public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false;
|
public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false;
|
||||||
|
@ -192,9 +197,16 @@ public class LogConfig extends AbstractConfig {
|
||||||
public static final String LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG = "leader.replication.throttled.replicas";
|
public static final String LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG = "leader.replication.throttled.replicas";
|
||||||
public static final String FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG = "follower.replication.throttled.replicas";
|
public static final String FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG = "follower.replication.throttled.replicas";
|
||||||
|
|
||||||
|
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
private static final String MESSAGE_FORMAT_VERSION_CONFIG = TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG;
|
private static final String MESSAGE_FORMAT_VERSION_CONFIG = TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG;
|
||||||
|
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
private static final String MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG = TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG;
|
||||||
|
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
private static final String MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC = TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC;
|
||||||
|
|
||||||
// Visible for testing
|
// Visible for testing
|
||||||
public static final Set<String> CONFIGS_WITH_NO_SERVER_DEFAULTS = Collections.unmodifiableSet(Utils.mkSet(
|
public static final Set<String> CONFIGS_WITH_NO_SERVER_DEFAULTS = Collections.unmodifiableSet(Utils.mkSet(
|
||||||
TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG,
|
TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG,
|
||||||
|
@ -261,8 +273,12 @@ public class LogConfig extends AbstractConfig {
|
||||||
MESSAGE_FORMAT_VERSION_DOC)
|
MESSAGE_FORMAT_VERSION_DOC)
|
||||||
.define(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, DEFAULT_MESSAGE_TIMESTAMP_TYPE,
|
.define(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, DEFAULT_MESSAGE_TIMESTAMP_TYPE,
|
||||||
in("CreateTime", "LogAppendTime"), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC)
|
in("CreateTime", "LogAppendTime"), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC)
|
||||||
.define(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS,
|
.define(MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS,
|
||||||
atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
|
atLeast(0), MEDIUM, MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
|
||||||
|
.define(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_BEFORE_MAX_MS,
|
||||||
|
atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC)
|
||||||
|
.define(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_AFTER_MAX_MS,
|
||||||
|
atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC)
|
||||||
.define(LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS,
|
.define(LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS,
|
||||||
ThrottledReplicaListValidator.INSTANCE, MEDIUM, LEADER_REPLICATION_THROTTLED_REPLICAS_DOC)
|
ThrottledReplicaListValidator.INSTANCE, MEDIUM, LEADER_REPLICATION_THROTTLED_REPLICAS_DOC)
|
||||||
.define(FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS,
|
.define(FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS,
|
||||||
|
@ -309,7 +325,12 @@ public class LogConfig extends AbstractConfig {
|
||||||
public final MetadataVersion messageFormatVersion;
|
public final MetadataVersion messageFormatVersion;
|
||||||
|
|
||||||
public final TimestampType messageTimestampType;
|
public final TimestampType messageTimestampType;
|
||||||
|
|
||||||
|
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details regarding the deprecation */
|
||||||
|
@Deprecated
|
||||||
public final long messageTimestampDifferenceMaxMs;
|
public final long messageTimestampDifferenceMaxMs;
|
||||||
|
public final long messageTimestampBeforeMaxMs;
|
||||||
|
public final long messageTimestampAfterMaxMs;
|
||||||
public final List<String> leaderReplicationThrottledReplicas;
|
public final List<String> leaderReplicationThrottledReplicas;
|
||||||
public final List<String> followerReplicationThrottledReplicas;
|
public final List<String> followerReplicationThrottledReplicas;
|
||||||
public final boolean messageDownConversionEnable;
|
public final boolean messageDownConversionEnable;
|
||||||
|
@ -358,6 +379,8 @@ public class LogConfig extends AbstractConfig {
|
||||||
this.messageFormatVersion = MetadataVersion.fromVersionString(getString(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG));
|
this.messageFormatVersion = MetadataVersion.fromVersionString(getString(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG));
|
||||||
this.messageTimestampType = TimestampType.forName(getString(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG));
|
this.messageTimestampType = TimestampType.forName(getString(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG));
|
||||||
this.messageTimestampDifferenceMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG);
|
this.messageTimestampDifferenceMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG);
|
||||||
|
this.messageTimestampBeforeMaxMs = getMessageTimestampBeforeMaxMs();
|
||||||
|
this.messageTimestampAfterMaxMs = getMessageTimestampAfterMaxMs();
|
||||||
this.leaderReplicationThrottledReplicas = Collections.unmodifiableList(getList(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG));
|
this.leaderReplicationThrottledReplicas = Collections.unmodifiableList(getList(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG));
|
||||||
this.followerReplicationThrottledReplicas = Collections.unmodifiableList(getList(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG));
|
this.followerReplicationThrottledReplicas = Collections.unmodifiableList(getList(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG));
|
||||||
this.messageDownConversionEnable = getBoolean(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG);
|
this.messageDownConversionEnable = getBoolean(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG);
|
||||||
|
@ -365,6 +388,30 @@ public class LogConfig extends AbstractConfig {
|
||||||
remoteLogConfig = new RemoteLogConfig(this);
|
remoteLogConfig = new RemoteLogConfig(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
//In the transition period before messageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
|
||||||
|
// we are using its value if messageTimestampBeforeMaxMs default value hasn't changed.
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
private long getMessageTimestampBeforeMaxMs() {
|
||||||
|
final Long messageTimestampBeforeMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG);
|
||||||
|
if (!messageTimestampBeforeMaxMs.equals(Long.MAX_VALUE)) {
|
||||||
|
return messageTimestampBeforeMaxMs;
|
||||||
|
} else {
|
||||||
|
return messageTimestampDifferenceMaxMs;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
//In the transition period before messageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
|
||||||
|
// we are using its value if messageTimestampAfterMaxMs default value hasn't changed.
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
private long getMessageTimestampAfterMaxMs() {
|
||||||
|
final Long messageTimestampAfterMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG);
|
||||||
|
if (!messageTimestampAfterMaxMs.equals(Long.MAX_VALUE)) {
|
||||||
|
return messageTimestampAfterMaxMs;
|
||||||
|
} else {
|
||||||
|
return messageTimestampDifferenceMaxMs;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public RecordVersion recordVersion() {
|
public RecordVersion recordVersion() {
|
||||||
return messageFormatVersion.highestSupportedRecordVersion();
|
return messageFormatVersion.highestSupportedRecordVersion();
|
||||||
|
|
|
@ -102,7 +102,8 @@ public class LogValidator {
|
||||||
private final boolean compactedTopic;
|
private final boolean compactedTopic;
|
||||||
private final byte toMagic;
|
private final byte toMagic;
|
||||||
private final TimestampType timestampType;
|
private final TimestampType timestampType;
|
||||||
private final long timestampDiffMaxMs;
|
private final long timestampBeforeMaxMs;
|
||||||
|
private final long timestampAfterMaxMs;
|
||||||
private final int partitionLeaderEpoch;
|
private final int partitionLeaderEpoch;
|
||||||
private final AppendOrigin origin;
|
private final AppendOrigin origin;
|
||||||
private final MetadataVersion interBrokerProtocolVersion;
|
private final MetadataVersion interBrokerProtocolVersion;
|
||||||
|
@ -115,7 +116,8 @@ public class LogValidator {
|
||||||
boolean compactedTopic,
|
boolean compactedTopic,
|
||||||
byte toMagic,
|
byte toMagic,
|
||||||
TimestampType timestampType,
|
TimestampType timestampType,
|
||||||
long timestampDiffMaxMs,
|
long timestampBeforeMaxMs,
|
||||||
|
long timestampAfterMaxMs,
|
||||||
int partitionLeaderEpoch,
|
int partitionLeaderEpoch,
|
||||||
AppendOrigin origin,
|
AppendOrigin origin,
|
||||||
MetadataVersion interBrokerProtocolVersion) {
|
MetadataVersion interBrokerProtocolVersion) {
|
||||||
|
@ -127,7 +129,8 @@ public class LogValidator {
|
||||||
this.compactedTopic = compactedTopic;
|
this.compactedTopic = compactedTopic;
|
||||||
this.toMagic = toMagic;
|
this.toMagic = toMagic;
|
||||||
this.timestampType = timestampType;
|
this.timestampType = timestampType;
|
||||||
this.timestampDiffMaxMs = timestampDiffMaxMs;
|
this.timestampBeforeMaxMs = timestampBeforeMaxMs;
|
||||||
|
this.timestampAfterMaxMs = timestampAfterMaxMs;
|
||||||
this.partitionLeaderEpoch = partitionLeaderEpoch;
|
this.partitionLeaderEpoch = partitionLeaderEpoch;
|
||||||
this.origin = origin;
|
this.origin = origin;
|
||||||
this.interBrokerProtocolVersion = interBrokerProtocolVersion;
|
this.interBrokerProtocolVersion = interBrokerProtocolVersion;
|
||||||
|
@ -208,7 +211,7 @@ public class LogValidator {
|
||||||
int batchIndex = 0;
|
int batchIndex = 0;
|
||||||
for (Record record : batch) {
|
for (Record record : batch) {
|
||||||
Optional<ApiRecordError> recordError = validateRecord(batch, topicPartition,
|
Optional<ApiRecordError> recordError = validateRecord(batch, topicPartition,
|
||||||
record, batchIndex, now, timestampType, timestampDiffMaxMs, compactedTopic,
|
record, batchIndex, now, timestampType, timestampBeforeMaxMs, timestampAfterMaxMs, compactedTopic,
|
||||||
metricsRecorder);
|
metricsRecorder);
|
||||||
recordError.ifPresent(e -> recordErrors.add(e));
|
recordError.ifPresent(e -> recordErrors.add(e));
|
||||||
// we fail the batch if any record fails, so we stop appending if any record fails
|
// we fail the batch if any record fails, so we stop appending if any record fails
|
||||||
|
@ -257,7 +260,7 @@ public class LogValidator {
|
||||||
int batchIndex = 0;
|
int batchIndex = 0;
|
||||||
for (Record record : batch) {
|
for (Record record : batch) {
|
||||||
Optional<ApiRecordError> recordError = validateRecord(batch, topicPartition, record,
|
Optional<ApiRecordError> recordError = validateRecord(batch, topicPartition, record,
|
||||||
batchIndex, now, timestampType, timestampDiffMaxMs, compactedTopic, metricsRecorder);
|
batchIndex, now, timestampType, timestampBeforeMaxMs, timestampAfterMaxMs, compactedTopic, metricsRecorder);
|
||||||
recordError.ifPresent(e -> recordErrors.add(e));
|
recordError.ifPresent(e -> recordErrors.add(e));
|
||||||
|
|
||||||
long offset = offsetCounter.value++;
|
long offset = offsetCounter.value++;
|
||||||
|
@ -368,7 +371,7 @@ public class LogValidator {
|
||||||
batchIndex, record);
|
batchIndex, record);
|
||||||
if (!recordError.isPresent()) {
|
if (!recordError.isPresent()) {
|
||||||
recordError = validateRecord(batch, topicPartition, record, batchIndex, now,
|
recordError = validateRecord(batch, topicPartition, record, batchIndex, now,
|
||||||
timestampType, timestampDiffMaxMs, compactedTopic, metricsRecorder);
|
timestampType, timestampBeforeMaxMs, timestampAfterMaxMs, compactedTopic, metricsRecorder);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!recordError.isPresent()
|
if (!recordError.isPresent()
|
||||||
|
@ -537,7 +540,8 @@ public class LogValidator {
|
||||||
int batchIndex,
|
int batchIndex,
|
||||||
long now,
|
long now,
|
||||||
TimestampType timestampType,
|
TimestampType timestampType,
|
||||||
long timestampDiffMaxMs,
|
long timestampBeforeMaxMs,
|
||||||
|
long timestampAfterMaxMs,
|
||||||
boolean compactedTopic,
|
boolean compactedTopic,
|
||||||
MetricsRecorder metricsRecorder) {
|
MetricsRecorder metricsRecorder) {
|
||||||
if (!record.hasMagic(batch.magic())) {
|
if (!record.hasMagic(batch.magic())) {
|
||||||
|
@ -566,7 +570,7 @@ public class LogValidator {
|
||||||
if (keyError.isPresent())
|
if (keyError.isPresent())
|
||||||
return keyError;
|
return keyError;
|
||||||
else
|
else
|
||||||
return validateTimestamp(batch, record, batchIndex, now, timestampType, timestampDiffMaxMs);
|
return validateTimestamp(batch, record, batchIndex, now, timestampType, timestampBeforeMaxMs, timestampAfterMaxMs);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Optional<ApiRecordError> validateKey(Record record,
|
private static Optional<ApiRecordError> validateKey(Record record,
|
||||||
|
@ -588,20 +592,29 @@ public class LogValidator {
|
||||||
int batchIndex,
|
int batchIndex,
|
||||||
long now,
|
long now,
|
||||||
TimestampType timestampType,
|
TimestampType timestampType,
|
||||||
long timestampDiffMaxMs) {
|
long timestampBeforeMaxMs,
|
||||||
if (timestampType == TimestampType.CREATE_TIME
|
long timestampAfterMaxMs) {
|
||||||
&& record.timestamp() != RecordBatch.NO_TIMESTAMP
|
if (timestampType == TimestampType.CREATE_TIME && record.timestamp() != RecordBatch.NO_TIMESTAMP) {
|
||||||
&& Math.abs(record.timestamp() - now) > timestampDiffMaxMs)
|
if (recordHasInvalidTimestamp(record, now, timestampBeforeMaxMs, timestampAfterMaxMs)) {
|
||||||
return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex,
|
return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex,
|
||||||
"Timestamp " + record.timestamp() + " of message with offset " + record.offset()
|
"Timestamp " + record.timestamp() + " of message with offset " + record.offset()
|
||||||
+ " is out of range. The timestamp should be within [" + (now - timestampDiffMaxMs)
|
+ " is out of range. The timestamp should be within [" + (now - timestampBeforeMaxMs)
|
||||||
+ ", " + (now + timestampDiffMaxMs) + "]")));
|
+ ", " + (now + timestampAfterMaxMs) + "]")));
|
||||||
else if (batch.timestampType() == TimestampType.LOG_APPEND_TIME)
|
}
|
||||||
|
} else if (batch.timestampType() == TimestampType.LOG_APPEND_TIME)
|
||||||
return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex,
|
return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex,
|
||||||
"Invalid timestamp type in message " + record + ". Producer should not set timestamp "
|
"Invalid timestamp type in message " + record + ". Producer should not set timestamp "
|
||||||
+ "type to LogAppendTime.")));
|
+ "type to LogAppendTime.")));
|
||||||
else
|
return Optional.empty();
|
||||||
return Optional.empty();
|
}
|
||||||
|
|
||||||
|
private static boolean recordHasInvalidTimestamp(Record record,
|
||||||
|
long now,
|
||||||
|
long timestampBeforeMaxMs,
|
||||||
|
long timestampAfterMaxMs) {
|
||||||
|
final long timestampDiff = now - record.timestamp();
|
||||||
|
return timestampDiff > timestampBeforeMaxMs ||
|
||||||
|
-1 * timestampDiff > timestampAfterMaxMs;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Optional<ApiRecordError> validateRecordCompression(CompressionType sourceCompression,
|
private static Optional<ApiRecordError> validateRecordCompression(CompressionType sourceCompression,
|
||||||
|
|
Loading…
Reference in New Issue