diff --git a/checkstyle/import-control-server-common.xml b/checkstyle/import-control-server-common.xml
index 5af34ca4a1e62..24a9cd3440160 100644
--- a/checkstyle/import-control-server-common.xml
+++ b/checkstyle/import-control-server-common.xml
@@ -111,9 +111,13 @@
+
+
+
+
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java
index 9c278cc45cc50..a33ac96162664 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java
@@ -94,6 +94,8 @@
import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG;
import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG;
import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG;
+import static org.apache.kafka.server.config.ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG;
+import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
/**
* Setup an embedded Kafka cluster with specified number of brokers and specified broker properties. To be used for
@@ -163,7 +165,7 @@ private void doStart() {
putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true);
putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0);
putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) brokers.length);
- putIfAbsent(brokerConfig, KafkaConfig.AutoCreateTopicsEnableProp(), false);
+ putIfAbsent(brokerConfig, AUTO_CREATE_TOPICS_ENABLE_CONFIG, false);
// reduce the size of the log cleaner map to reduce test memory usage
putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L);
@@ -177,7 +179,7 @@ private void doStart() {
for (int i = 0; i < brokers.length; i++) {
brokerConfig.put(KafkaConfig.BrokerIdProp(), i);
currentBrokerLogDirs[i] = currentBrokerLogDirs[i] == null ? createLogDir() : currentBrokerLogDirs[i];
- brokerConfig.put(KafkaConfig.LogDirProp(), currentBrokerLogDirs[i]);
+ brokerConfig.put(LOG_DIR_CONFIG, currentBrokerLogDirs[i]);
if (!hasListenerConfig)
brokerConfig.put(KafkaConfig.ListenersProp(), listenerName.value() + "://localhost:" + currentBrokerPorts[i]);
brokers[i] = TestUtils.createServer(new KafkaConfig(brokerConfig, true), time);
diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java
index 27a4af1468ac0..d40ba17cc2741 100644
--- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java
+++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java
@@ -120,6 +120,7 @@
import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX;
import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC;
+import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
/**
* This class is responsible for
@@ -282,7 +283,7 @@ private void configureRLMM() {
rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps());
rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId);
- rlmmProps.put(KafkaConfig.LogDirProp(), logDir);
+ rlmmProps.put(LOG_DIR_CONFIG, logDir);
rlmmProps.put("cluster.id", clusterId);
remoteLogMetadataManager.configure(rlmmProps);
diff --git a/core/src/main/scala/kafka/MetadataLogConfig.scala b/core/src/main/scala/kafka/MetadataLogConfig.scala
index 60afac20fa4d5..544f688d8a2f5 100755
--- a/core/src/main/scala/kafka/MetadataLogConfig.scala
+++ b/core/src/main/scala/kafka/MetadataLogConfig.scala
@@ -18,7 +18,7 @@ package kafka.raft
import kafka.server.KafkaConfig
import org.apache.kafka.common.config.AbstractConfig
-import org.apache.kafka.storage.internals.log.LogConfig
+import org.apache.kafka.server.config.ServerLogConfigs
final case class MetadataLogConfig(
logSegmentBytes: Int,
@@ -42,7 +42,7 @@ object MetadataLogConfig {
config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp),
maxBatchSizeInBytes,
maxFetchSizeInBytes,
- LogConfig.DEFAULT_FILE_DELETE_DELAY_MS,
+ ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
config.getInt(KafkaConfig.NodeIdProp)
)
}
diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
index 514b08797a10a..ab96349a3426e 100644
--- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
+++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
@@ -31,6 +31,7 @@ import org.apache.kafka.common.record.{MemoryRecords, Records}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch}
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals
@@ -553,7 +554,7 @@ object KafkaMetadataLog extends Logging {
props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString)
props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString)
props.setProperty(TopicConfig.SEGMENT_MS_CONFIG, config.logSegmentMillis.toString)
- props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS.toString)
+ props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString)
// Disable time and byte retention when deleting segments
props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1")
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index ac74eb858c38d..a2be1870b1b8e 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -21,7 +21,6 @@ import kafka.metrics.LinuxIoMetricsCollector
import kafka.migration.MigrationPropagator
import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.raft.KafkaRaftManager
-import kafka.server.KafkaConfig.{AlterConfigPolicyClassNameProp, CreateTopicPolicyClassNameProp}
import kafka.server.QuotaFactory.QuotaManagers
import scala.collection.immutable
@@ -46,6 +45,7 @@ import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.security.{CredentialProvider, PasswordEncoder}
import org.apache.kafka.server.NodeToControllerChannelManager
import org.apache.kafka.server.authorizer.Authorizer
+import org.apache.kafka.server.config.ServerLogConfigs.{ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG}
import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.server.config.ConfigType
import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
@@ -207,9 +207,9 @@ class ControllerServer(
sharedServer.startForController()
createTopicPolicy = Option(config.
- getConfiguredInstance(CreateTopicPolicyClassNameProp, classOf[CreateTopicPolicy]))
+ getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy]))
alterConfigPolicy = Option(config.
- getConfiguredInstance(AlterConfigPolicyClassNameProp, classOf[AlterConfigPolicy]))
+ getConfiguredInstance(ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[AlterConfigPolicy]))
val voterConnections = FutureUtils.waitWithLogging(logger.underlying, logIdent,
"controller quorum voters future",
diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index 6f7319180d7ea..81fc98479cec7 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -39,8 +39,8 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals}
-import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.ClientMetricsReceiverPlugin
import org.apache.kafka.server.telemetry.ClientTelemetry
import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig}
@@ -120,14 +120,14 @@ object DynamicBrokerConfig {
def brokerConfigSynonyms(name: String, matchListenerOverride: Boolean): List[String] = {
name match {
- case KafkaConfig.LogRollTimeMillisProp | KafkaConfig.LogRollTimeHoursProp =>
- List(KafkaConfig.LogRollTimeMillisProp, KafkaConfig.LogRollTimeHoursProp)
- case KafkaConfig.LogRollTimeJitterMillisProp | KafkaConfig.LogRollTimeJitterHoursProp =>
- List(KafkaConfig.LogRollTimeJitterMillisProp, KafkaConfig.LogRollTimeJitterHoursProp)
- case KafkaConfig.LogFlushIntervalMsProp => // LogFlushSchedulerIntervalMsProp is used as default
- List(KafkaConfig.LogFlushIntervalMsProp, KafkaConfig.LogFlushSchedulerIntervalMsProp)
- case KafkaConfig.LogRetentionTimeMillisProp | KafkaConfig.LogRetentionTimeMinutesProp | KafkaConfig.LogRetentionTimeHoursProp =>
- List(KafkaConfig.LogRetentionTimeMillisProp, KafkaConfig.LogRetentionTimeMinutesProp, KafkaConfig.LogRetentionTimeHoursProp)
+ case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG | ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG =>
+ List(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG)
+ case ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG | ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG =>
+ List(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG)
+ case ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG => // KafkaLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG is used as default
+ List(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG)
+ case ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG | ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG | ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG =>
+ List(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG)
case ListenerConfigRegex(baseName) if matchListenerOverride =>
// `ListenerMechanismConfigs` are specified as listenerPrefix.mechanism.
// and other listener configs are specified as listenerPrefix.
@@ -674,7 +674,7 @@ object DynamicLogConfig {
// Exclude message.format.version for now since we need to check that the version
// is supported on all brokers in the cluster.
@nowarn("cat=deprecation")
- val ExcludedConfigs = Set(KafkaConfig.LogMessageFormatVersionProp)
+ val ExcludedConfigs = Set(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG)
val ReconfigurableConfigs = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.values.asScala.toSet -- ExcludedConfigs
val KafkaConfigToLogConfigName = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) }
@@ -697,11 +697,11 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok
if (logRetentionMs != -1L && logLocalRetentionMs != -2L) {
if (logLocalRetentionMs == -1L) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs,
- s"Value must not be -1 as ${KafkaConfig.LogRetentionTimeMillisProp} value is set as $logRetentionMs.")
+ s"Value must not be -1 as ${ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG} value is set as $logRetentionMs.")
}
if (logLocalRetentionMs > logRetentionMs) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs,
- s"Value must not be more than ${KafkaConfig.LogRetentionTimeMillisProp} property value: $logRetentionMs")
+ s"Value must not be more than ${ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG} property value: $logRetentionMs")
}
}
}
@@ -712,11 +712,11 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok
if (logRetentionBytes > -1 && logLocalRetentionBytes != -2) {
if (logLocalRetentionBytes == -1) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes,
- s"Value must not be -1 as ${KafkaConfig.LogRetentionBytesProp} value is set as $logRetentionBytes.")
+ s"Value must not be -1 as ${ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG} value is set as $logRetentionBytes.")
}
if (logLocalRetentionBytes > logRetentionBytes) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes,
- s"Value must not be more than ${KafkaConfig.LogRetentionBytesProp} property value: $logRetentionBytes")
+ s"Value must not be more than ${ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG} property value: $logRetentionBytes")
}
}
}
@@ -771,7 +771,7 @@ object DynamicThreadPool {
val ReconfigurableConfigs = Set(
KafkaConfig.NumIoThreadsProp,
ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG,
- KafkaConfig.NumRecoveryThreadsPerDataDirProp,
+ ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG,
KafkaConfig.BackgroundThreadsProp)
def validateReconfiguration(currentConfig: KafkaConfig, newConfig: KafkaConfig): Unit = {
@@ -796,7 +796,7 @@ object DynamicThreadPool {
name match {
case KafkaConfig.NumIoThreadsProp => config.numIoThreads
case ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG => config.numReplicaFetchers
- case KafkaConfig.NumRecoveryThreadsPerDataDirProp => config.numRecoveryThreadsPerDataDir
+ case ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG => config.numRecoveryThreadsPerDataDir
case KafkaConfig.BackgroundThreadsProp => config.backgroundThreads
case n => throw new IllegalStateException(s"Unexpected config $n")
}
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 5adfae61bc76b..6c23eb667cf04 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -47,8 +47,7 @@ import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator}
import org.apache.kafka.server.common.MetadataVersion._
-import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.record.BrokerCompressionType
import org.apache.kafka.server.util.Csv
@@ -63,8 +62,6 @@ import scala.collection.{Map, Seq}
object KafkaConfig {
- private val LogConfigPrefix = "log."
-
def main(args: Array[String]): Unit = {
System.out.println(configDef.toHtml(4, (config: String) => "brokerconfigs_" + config,
DynamicBrokerConfig.dynamicConfigUpdateModes))
@@ -157,54 +154,7 @@ object KafkaConfig {
val FailedAuthenticationDelayMsProp = "connection.failed.authentication.delay.ms"
/***************** rack configuration *************/
val RackProp = "broker.rack"
- /** ********* Log Configuration ***********/
- val NumPartitionsProp = "num.partitions"
- val LogDirsProp = LogConfigPrefix + "dirs"
- val LogDirProp = LogConfigPrefix + "dir"
- val LogSegmentBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG)
-
- val LogRollTimeMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG)
- val LogRollTimeHoursProp = LogConfigPrefix + "roll.hours"
-
- val LogRollTimeJitterMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_JITTER_MS_CONFIG)
- val LogRollTimeJitterHoursProp = LogConfigPrefix + "roll.jitter.hours"
-
- val LogRetentionTimeMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_MS_CONFIG)
- val LogRetentionTimeMinutesProp = LogConfigPrefix + "retention.minutes"
- val LogRetentionTimeHoursProp = LogConfigPrefix + "retention.hours"
-
- val LogRetentionBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_BYTES_CONFIG)
- val LogCleanupIntervalMsProp = LogConfigPrefix + "retention.check.interval.ms"
- val LogCleanupPolicyProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG)
- val LogIndexSizeMaxBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG)
- val LogIndexIntervalBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG)
- val LogFlushIntervalMessagesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG)
- val LogDeleteDelayMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG)
- val LogFlushSchedulerIntervalMsProp = LogConfigPrefix + "flush.scheduler.interval.ms"
- val LogFlushIntervalMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MS_CONFIG)
- val LogFlushOffsetCheckpointIntervalMsProp = LogConfigPrefix + "flush.offset.checkpoint.interval.ms"
- val LogFlushStartOffsetCheckpointIntervalMsProp = LogConfigPrefix + "flush.start.offset.checkpoint.interval.ms"
- val LogPreAllocateProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.PREALLOCATE_CONFIG)
-
- /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
- @deprecated("3.0")
- val LogMessageFormatVersionProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_FORMAT_VERSION_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 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 AutoCreateTopicsEnableProp = "auto.create.topics.enable"
- val MinInSyncReplicasProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG)
- val CreateTopicPolicyClassNameProp = "create.topic.policy.class.name"
- val AlterConfigPolicyClassNameProp = "alter.config.policy.class.name"
- val LogMessageDownConversionEnableProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
/** ********* Controlled shutdown configuration ***********/
val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
@@ -405,75 +355,6 @@ object KafkaConfig {
s"This must be configured to be less than $ConnectionsMaxIdleMsProp to prevent connection timeout."
/************* Rack Configuration **************/
val RackDoc = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: RACK1
, us-east-1d
"
- /** ********* Log Configuration ***********/
- val NumPartitionsDoc = "The default number of log partitions per topic"
- val LogDirDoc = "The directory in which the log data is kept (supplemental for " + LogDirsProp + " property)"
- val LogDirsDoc = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LogDirProp + " is used."
- val LogSegmentBytesDoc = "The maximum size of a single log file"
- val LogRollTimeMillisDoc = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LogRollTimeHoursProp + " is used"
- val LogRollTimeHoursDoc = "The maximum time before a new log segment is rolled out (in hours), secondary to " + LogRollTimeMillisProp + " property"
-
- val LogRollTimeJitterMillisDoc = "The maximum jitter to subtract from logRollTimeMillis (in milliseconds). If not set, the value in " + LogRollTimeJitterHoursProp + " is used"
- val LogRollTimeJitterHoursDoc = "The maximum jitter to subtract from logRollTimeMillis (in hours), secondary to " + LogRollTimeJitterMillisProp + " property"
-
- val LogRetentionTimeMillisDoc = "The number of milliseconds to keep a log file before deleting it (in milliseconds), If not set, the value in " + LogRetentionTimeMinutesProp + " is used. If set to -1, no time limit is applied."
- val LogRetentionTimeMinsDoc = "The number of minutes to keep a log file before deleting it (in minutes), secondary to " + LogRetentionTimeMillisProp + " property. If not set, the value in " + LogRetentionTimeHoursProp + " is used"
- val LogRetentionTimeHoursDoc = "The number of hours to keep a log file before deleting it (in hours), tertiary to " + LogRetentionTimeMillisProp + " property"
-
- val LogRetentionBytesDoc = "The maximum size of the log before deleting it"
- val LogCleanupIntervalMsDoc = "The frequency in milliseconds that the log cleaner checks whether any log is eligible for deletion"
- val LogCleanupPolicyDoc = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies. Valid policies are: \"delete\" and \"compact\""
- val LogIndexSizeMaxBytesDoc = "The maximum size in bytes of the offset index"
- val LogIndexIntervalBytesDoc = "The interval with which we add an entry to the offset index."
- val LogFlushIntervalMessagesDoc = "The number of messages accumulated on a log partition before messages are flushed to disk."
- val LogDeleteDelayMsDoc = "The amount of time to wait before deleting a file from the filesystem"
- val LogFlushSchedulerIntervalMsDoc = "The frequency in ms that the log flusher checks whether any log needs to be flushed to disk"
- val LogFlushIntervalMsDoc = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk. If not set, the value in " + LogFlushSchedulerIntervalMsProp + " is used"
- val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point."
- val LogFlushStartOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of log start offset"
- val LogPreAllocateEnableDoc = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true."
- val LogMessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid MetadataVersion. " +
- "Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check MetadataVersion for more details. By setting a particular message format version, the " +
- "user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " +
- "will cause consumers with older versions to break as they will receive messages with a format that they don't understand."
-
- 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
."
-
- 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 " +
- "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."
-
- 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 AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server."
- val MinInSyncReplicasDoc = "When a producer sets acks to \"all\" (or \"-1\"), " +
- "min.insync.replicas
specifies the minimum number of replicas that must acknowledge " +
- "a write for the write to be considered successful. If this minimum cannot be met, " +
- "then the producer will raise an exception (either NotEnoughReplicas
or " +
- "NotEnoughReplicasAfterAppend
).
When used together, min.insync.replicas
and acks " +
- "allow you to enforce greater durability guarantees. A typical scenario would be to " +
- "create a topic with a replication factor of 3, set min.insync.replicas
to 2, and " +
- "produce with acks of \"all\". This will ensure that the producer raises an exception " +
- "if a majority of replicas do not receive a write."
-
- val CreateTopicPolicyClassNameDoc = "The create topic policy class that should be used for validation. The class should " +
- "implement the org.apache.kafka.server.policy.CreateTopicPolicy
interface."
- val AlterConfigPolicyClassNameDoc = "The alter configs policy class that should be used for validation. The class should " +
- "implement the org.apache.kafka.server.policy.AlterConfigPolicy
interface."
- val LogMessageDownConversionEnableDoc = TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC
/** ********* Controlled shutdown configuration ***********/
val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens"
@@ -643,24 +524,24 @@ object KafkaConfig {
.define(RackProp, STRING, null, MEDIUM, RackDoc)
/** ********* Log Configuration ***********/
- .define(NumPartitionsProp, INT, Defaults.NUM_PARTITIONS, atLeast(1), MEDIUM, NumPartitionsDoc)
- .define(LogDirProp, STRING, Defaults.LOG_DIR, HIGH, LogDirDoc)
- .define(LogDirsProp, STRING, null, HIGH, LogDirsDoc)
- .define(LogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, LogSegmentBytesDoc)
+ .define(ServerLogConfigs.NUM_PARTITIONS_CONFIG, INT, ServerLogConfigs.NUM_PARTITIONS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.NUM_PARTITIONS_DOC)
+ .define(ServerLogConfigs.LOG_DIR_CONFIG, STRING, ServerLogConfigs.LOG_DIR_DEFAULT, HIGH, ServerLogConfigs.LOG_DIR_DOC)
+ .define(ServerLogConfigs.LOG_DIRS_CONFIG, STRING, null, HIGH, ServerLogConfigs.LOG_DIRS_DOC)
+ .define(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC)
- .define(LogRollTimeMillisProp, LONG, null, HIGH, LogRollTimeMillisDoc)
- .define(LogRollTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_MS).toInt, atLeast(1), HIGH, LogRollTimeHoursDoc)
+ .define(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_ROLL_TIME_MILLIS_DOC)
+ .define(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_MS).toInt, atLeast(1), HIGH, ServerLogConfigs.LOG_ROLL_TIME_HOURS_DOC)
- .define(LogRollTimeJitterMillisProp, LONG, null, HIGH, LogRollTimeJitterMillisDoc)
- .define(LogRollTimeJitterHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_JITTER_MS).toInt, atLeast(0), HIGH, LogRollTimeJitterHoursDoc)
+ .define(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_DOC)
+ .define(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_JITTER_MS).toInt, atLeast(0), HIGH, ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_DOC)
- .define(LogRetentionTimeMillisProp, LONG, null, HIGH, LogRetentionTimeMillisDoc)
- .define(LogRetentionTimeMinutesProp, INT, null, HIGH, LogRetentionTimeMinsDoc)
- .define(LogRetentionTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_RETENTION_MS).toInt, HIGH, LogRetentionTimeHoursDoc)
+ .define(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_DOC)
+ .define(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, INT, null, HIGH, ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_DOC)
+ .define(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_RETENTION_MS).toInt, HIGH, ServerLogConfigs.LOG_RETENTION_TIME_HOURS_DOC)
- .define(LogRetentionBytesProp, LONG, LogConfig.DEFAULT_RETENTION_BYTES, HIGH, LogRetentionBytesDoc)
- .define(LogCleanupIntervalMsProp, LONG, Defaults.LOG_CLEANUP_INTERVAL_MS, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc)
- .define(LogCleanupPolicyProp, LIST, LogConfig.DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, LogCleanupPolicyDoc)
+ .define(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, LONG, ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, HIGH, ServerLogConfigs.LOG_RETENTION_BYTES_DOC)
+ .define(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG, LONG, ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_DOC)
+ .define(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG, LIST, ServerLogConfigs.LOG_CLEANUP_POLICY_DEFAULT, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, ServerLogConfigs.LOG_CLEANUP_POLICY_DOC)
.define(CleanerConfig.LOG_CLEANER_THREADS_PROP, INT, CleanerConfig.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_THREADS_DOC)
.define(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, DOUBLE, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_DOC)
.define(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, LONG, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_DOC)
@@ -672,26 +553,26 @@ object KafkaConfig {
.define(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_DOC)
.define(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_DOC)
.define(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_DOC)
- .define(LogIndexSizeMaxBytesProp, INT, LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc)
- .define(LogIndexIntervalBytesProp, INT, LogConfig.DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc)
- .define(LogFlushIntervalMessagesProp, LONG, LogConfig.DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), HIGH, LogFlushIntervalMessagesDoc)
- .define(LogDeleteDelayMsProp, LONG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), HIGH, LogDeleteDelayMsDoc)
- .define(LogFlushSchedulerIntervalMsProp, LONG, LogConfig.DEFAULT_FLUSH_MS, HIGH, LogFlushSchedulerIntervalMsDoc)
- .define(LogFlushIntervalMsProp, LONG, null, HIGH, LogFlushIntervalMsDoc)
- .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc)
- .define(LogFlushStartOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushStartOffsetCheckpointIntervalMsDoc)
- .define(LogPreAllocateProp, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, LogPreAllocateEnableDoc)
- .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NUM_RECOVERY_THREADS_PER_DATA_DIR, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc)
- .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AUTO_CREATE_TOPICS_ENABLE, HIGH, AutoCreateTopicsEnableDoc)
- .define(MinInSyncReplicasProp, INT, LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS, atLeast(1), HIGH, MinInSyncReplicasDoc)
- .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(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(AlterConfigPolicyClassNameProp, CLASS, null, LOW, AlterConfigPolicyClassNameDoc)
- .define(LogMessageDownConversionEnableProp, BOOLEAN, LogConfig.DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, LogMessageDownConversionEnableDoc)
+ .define(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT, atLeast(4), MEDIUM, ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DOC)
+ .define(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DOC)
+ .define(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_DOC)
+ .define(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, atLeast(0), HIGH, ServerLogConfigs.LOG_DELETE_DELAY_MS_DOC)
+ .define(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_DEFAULT, HIGH, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_DOC)
+ .define(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_DOC)
+ .define(ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG, INT, ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DOC)
+ .define(ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG, INT, ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DOC)
+ .define(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, ServerLogConfigs.LOG_PRE_ALLOCATE_ENABLE_DOC)
+ .define(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, INT, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_DOC)
+ .define(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DEFAULT, HIGH, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DOC)
+ .define(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, INT, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DOC)
+ .define(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT, new MetadataVersionValidator(), MEDIUM, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DOC)
+ .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT, ConfigDef.ValidString.in("CreateTime", "LogAppendTime"), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DOC)
+ .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
+ .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC)
+ .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC)
+ .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC)
+ .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC)
+ .define(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT, LOW, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DOC)
/** ********* Replication configuration ***********/
.define(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, INT, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DEFAULT, MEDIUM, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DOC)
@@ -1235,21 +1116,22 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG))
/** ********* Log Configuration ***********/
- val autoCreateTopicsEnable = getBoolean(KafkaConfig.AutoCreateTopicsEnableProp)
- val numPartitions = getInt(KafkaConfig.NumPartitionsProp)
- val logDirs = CoreUtils.parseCsvList(Option(getString(KafkaConfig.LogDirsProp)).getOrElse(getString(KafkaConfig.LogDirProp)))
- def logSegmentBytes = getInt(KafkaConfig.LogSegmentBytesProp)
- def logFlushIntervalMessages = getLong(KafkaConfig.LogFlushIntervalMessagesProp)
+ val autoCreateTopicsEnable = getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG)
+ val numPartitions = getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG)
+ val logDirs = CoreUtils.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG)))
+ def logSegmentBytes = getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG)
+ def logFlushIntervalMessages = getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG)
val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP)
- def numRecoveryThreadsPerDataDir = getInt(KafkaConfig.NumRecoveryThreadsPerDataDirProp)
- val logFlushSchedulerIntervalMs = getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)
- val logFlushOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushOffsetCheckpointIntervalMsProp).toLong
- val logFlushStartOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushStartOffsetCheckpointIntervalMsProp).toLong
- val logCleanupIntervalMs = getLong(KafkaConfig.LogCleanupIntervalMsProp)
- def logCleanupPolicy = getList(KafkaConfig.LogCleanupPolicyProp)
+ def numRecoveryThreadsPerDataDir = getInt(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG)
+ val logFlushSchedulerIntervalMs = getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG)
+ val logFlushOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
+ val logFlushStartOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
+ val logCleanupIntervalMs = getLong(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG)
+ def logCleanupPolicy = getList(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG)
+
val offsetsRetentionMinutes = getInt(GroupCoordinatorConfig.OFFSETS_RETENTION_MINUTES_CONFIG)
val offsetsRetentionCheckIntervalMs = getLong(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG)
- def logRetentionBytes = getLong(KafkaConfig.LogRetentionBytesProp)
+ def logRetentionBytes = getLong(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG)
val logCleanerDedupeBufferSize = getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP)
val logCleanerDedupeBufferLoadFactor = getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP)
val logCleanerIoBufferSize = getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP)
@@ -1260,40 +1142,40 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP)
def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP)
val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP)
- def logIndexSizeMaxBytes = getInt(KafkaConfig.LogIndexSizeMaxBytesProp)
- def logIndexIntervalBytes = getInt(KafkaConfig.LogIndexIntervalBytesProp)
- def logDeleteDelayMs = getLong(KafkaConfig.LogDeleteDelayMsProp)
- def logRollTimeMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeHoursProp))
- def logRollTimeJitterMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeJitterMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeJitterHoursProp))
- def logFlushIntervalMs: java.lang.Long = Option(getLong(KafkaConfig.LogFlushIntervalMsProp)).getOrElse(getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp))
- def minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp)
- def logPreAllocateEnable: java.lang.Boolean = getBoolean(KafkaConfig.LogPreAllocateProp)
+ def logIndexSizeMaxBytes = getInt(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG)
+ def logIndexIntervalBytes = getInt(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG)
+ def logDeleteDelayMs = getLong(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG)
+ def logRollTimeMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG))
+ def logRollTimeJitterMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG))
+ def logFlushIntervalMs: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG)).getOrElse(getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG))
+ def minInSyncReplicas = getInt(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG)
+ def logPreAllocateEnable: java.lang.Boolean = getBoolean(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG)
// We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
// is passed, `0.10.0-IV0` may be picked)
@nowarn("cat=deprecation")
- private val logMessageFormatVersionString = getString(KafkaConfig.LogMessageFormatVersionProp)
+ private val logMessageFormatVersionString = getString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG)
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
@deprecated("3.0")
lazy val logMessageFormatVersion =
if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion))
- MetadataVersion.fromVersionString(LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION)
+ MetadataVersion.fromVersionString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT)
else MetadataVersion.fromVersionString(logMessageFormatVersionString)
- def logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp))
+ def logMessageTimestampType = TimestampType.forName(getString(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG))
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
@deprecated("3.6")
- def logMessageTimestampDifferenceMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)
+ def logMessageTimestampDifferenceMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG)
// 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) {
+ val messageTimestampBeforeMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)
+ if (messageTimestampBeforeMaxMs != ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT) {
messageTimestampBeforeMaxMs
} else {
logMessageTimestampDifferenceMaxMs
@@ -1305,7 +1187,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
@nowarn("cat=deprecation")
def logMessageTimestampAfterMaxMs: Long = {
- val messageTimestampAfterMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampAfterMaxMsProp)
+ val messageTimestampAfterMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)
if (messageTimestampAfterMaxMs != Long.MaxValue) {
messageTimestampAfterMaxMs
} else {
@@ -1313,7 +1195,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
}
}
- def logMessageDownConversionEnable: Boolean = getBoolean(KafkaConfig.LogMessageDownConversionEnableProp)
+ def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
/** ********* Replication configuration ***********/
val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG)
@@ -1518,10 +1400,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val millisInHour = 60L * millisInMinute
val millis: java.lang.Long =
- Option(getLong(KafkaConfig.LogRetentionTimeMillisProp)).getOrElse(
- Option(getInt(KafkaConfig.LogRetentionTimeMinutesProp)) match {
+ Option(getLong(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)).getOrElse(
+ Option(getInt(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG)) match {
case Some(mins) => millisInMinute * mins
- case None => getInt(KafkaConfig.LogRetentionTimeHoursProp) * millisInHour
+ case None => getInt(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG) * millisInHour
})
if (millis < 0) return -1
@@ -1938,7 +1820,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@nowarn("cat=deprecation")
private def createBrokerWarningMessage: String = {
- s"Broker configuration ${KafkaConfig.LogMessageFormatVersionProp} with value $logMessageFormatVersionString is ignored " +
+ s"Broker configuration ${ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG} with value $logMessageFormatVersionString is ignored " +
s"because the inter-broker protocol version `$interBrokerProtocolVersionString` is greater or equal than 3.0. " +
"This configuration is deprecated and it will be removed in Apache Kafka 4.0."
}
diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
index 4ae5a8b023790..51f5d5b26d2fd 100644
--- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
@@ -123,7 +123,7 @@ object KafkaRaftServer {
/**
* Initialize the configured log directories, including both [[KafkaConfig.MetadataLogDirProp]]
- * and [[KafkaConfig.LogDirProp]]. This method performs basic validation to ensure that all
+ * and [[KafkaConfig.LOG_DIR_PROP]]. This method performs basic validation to ensure that all
* directories are accessible and have been initialized with consistent `meta.properties`.
*
* @param config The process configuration
diff --git a/core/src/main/scala/kafka/server/ZkAdminManager.scala b/core/src/main/scala/kafka/server/ZkAdminManager.scala
index 6acb4c518ceef..351fea4b81b10 100644
--- a/core/src/main/scala/kafka/server/ZkAdminManager.scala
+++ b/core/src/main/scala/kafka/server/ZkAdminManager.scala
@@ -49,6 +49,8 @@ import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, S
import org.apache.kafka.common.utils.Sanitizer
import org.apache.kafka.server.common.AdminOperationException
import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals}
+import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG
+import org.apache.kafka.server.config.ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG
import org.apache.kafka.storage.internals.log.LogConfig
import scala.collection.{Map, mutable, _}
@@ -79,10 +81,10 @@ class ZkAdminManager(val config: KafkaConfig,
private val configHelper = new ConfigHelper(metadataCache, config, new ZkConfigRepository(adminZkClient))
private val createTopicPolicy =
- Option(config.getConfiguredInstance(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[CreateTopicPolicy]))
+ Option(config.getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy]))
private val alterConfigPolicy =
- Option(config.getConfiguredInstance(KafkaConfig.AlterConfigPolicyClassNameProp, classOf[AlterConfigPolicy]))
+ Option(config.getConfiguredInstance(ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[AlterConfigPolicy]))
def hasDelayedTopicOperations = topicPurgatory.numDelayed != 0
diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
index a6c6a28011478..a40070793ccca 100644
--- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
+++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
@@ -73,6 +73,7 @@
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIRS_CONFIG;
import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG;
@@ -171,11 +172,11 @@ private KafkaConfig createNodeConfig(TestKitNode node) {
}
if (brokerNode != null) {
// Set the log.dirs according to the broker node setting (if there is a broker node)
- props.put(KafkaConfig$.MODULE$.LogDirsProp(),
+ props.put(LOG_DIRS_CONFIG,
String.join(",", brokerNode.logDataDirectories()));
} else {
// Set log.dirs equal to the metadata directory if there is just a controller.
- props.put(KafkaConfig$.MODULE$.LogDirsProp(),
+ props.put(LOG_DIRS_CONFIG,
controllerNode.metadataDirectory());
}
props.put(KafkaConfig$.MODULE$.ListenerSecurityProtocolMapProp(),
diff --git a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
index 62dbd1a0e393b..342ff3ab8e7fc 100644
--- a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
@@ -28,6 +28,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.requests.ListOffsetsResponse
import org.apache.kafka.common.utils.{MockTime, Time, Utils}
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -345,8 +346,8 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
}
// We use mock timer so the records can get removed if the test env is too busy to complete
// tests before kafka-log-retention. Hence, we disable the retention to avoid failed tests
- props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "-1")
- props.setProperty(KafkaConfig.LogDirProp, dataFolder(index))
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "-1")
+ props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, dataFolder(index))
props
}.map(KafkaConfig.fromProps)
}
diff --git a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala
index e7f3282431d83..dc3ff0fe45471 100644
--- a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala
+++ b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala
@@ -24,6 +24,7 @@ import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.config.{ConfigException, ConfigResource, TopicConfig}
import org.apache.kafka.common.errors.{InvalidConfigurationException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.utils.MockTime
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentState}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.function.Executable
@@ -387,9 +388,9 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, storageManagerClassName)
props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, metadataManagerClassName)
- props.put(KafkaConfig.LogRetentionTimeMillisProp, "2000")
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "2000")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "1000")
- props.put(KafkaConfig.LogRetentionBytesProp, "2048")
+ props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "2048")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "1024")
props
}
diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala
index 203e43b18481b..3c0e4106b3cf0 100644
--- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala
@@ -24,7 +24,7 @@ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp,
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException}
import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.server.config.KafkaSecurityConfigs
+import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs}
import org.apache.kafka.server.policy.AlterConfigPolicy
import org.apache.kafka.storage.internals.log.LogConfig
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue}
@@ -76,7 +76,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
}
private def overrideNodeConfigs(props: Properties): Unit = {
- props.put(KafkaConfig.AlterConfigPolicyClassNameProp, classOf[Policy])
+ props.put(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[Policy])
}
@ParameterizedTest
@@ -167,7 +167,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
assertEquals(4, configs.size)
assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
- assertEquals(LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value)
+ assertEquals(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value)
assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
@@ -199,7 +199,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
assertEquals(4, configs.size)
assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
- assertEquals(LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value)
+ assertEquals(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value)
assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala
index 3a24bffcc4852..bcfa7f2aa6c24 100644
--- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala
@@ -31,7 +31,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.config.KafkaSecurityConfigs
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout}
@@ -198,9 +198,9 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
// verify that they show up in the "configs" output of CreateTopics.
if (testInfo.getTestMethod.toString.contains("testCreateTopicsReturnsConfigs")) {
configs.foreach(config => {
- config.setProperty(KafkaConfig.LogRollTimeHoursProp, "2")
- config.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "240")
- config.setProperty(KafkaConfig.LogRollTimeJitterMillisProp, "123")
+ config.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "2")
+ config.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "240")
+ config.setProperty(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, "123")
})
}
configs.foreach { config =>
diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
index 02902b56f7a98..c1bf5660cce1b 100644
--- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
@@ -33,6 +33,7 @@ import org.apache.kafka.common.network.{ListenerName, Mode}
import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -47,7 +48,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
def generateConfigs: scala.collection.Seq[KafkaConfig] = {
val overridingProps = new Properties()
val numServers = 2
- overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString)
+ overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 4.toString)
TestUtils.createBrokerConfigs(
numServers,
zkConnectOrNull,
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
index 8fb981660019d..b5c0912cff2ff 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
@@ -26,7 +26,7 @@ import org.apache.kafka.common.message.FindCoordinatorRequestData
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinatorResponse}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.ShutdownableThread
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, Disabled, Test}
@@ -61,7 +61,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
properties.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0")
properties.put(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG, maxGroupSize)
properties.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true")
- properties.put(KafkaConfig.AutoCreateTopicsEnableProp, "false")
+ properties.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
FixedPortTestUtils.createBrokerConfigs(brokerCount, zkConnect, enableControlledShutdown = false)
.map(KafkaConfig.fromProps(_, properties))
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala
index 86de9a5e01eea..d01476f999df1 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala
@@ -27,6 +27,8 @@ import kafka.utils.{EmptyTestInfo, TestUtils}
import org.apache.kafka.clients.admin.NewTopic
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
+import org.apache.kafka.server.config.ServerLogConfigs
+
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
@@ -56,7 +58,7 @@ object ConsumerTopicCreationTest {
// configure server properties
this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown
- this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, brokerAutoTopicCreationEnable.toString)
+ this.serverConfig.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, brokerAutoTopicCreationEnable.toString)
// configure client properties
this.producerConfig.setProperty(ProducerConfig.CLIENT_ID_CONFIG, producerClientId)
diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
index 22bd16d5b4c74..6b9da732ed7f9 100644
--- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
@@ -40,7 +40,7 @@ import org.apache.kafka.common.security.auth._
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout}
@@ -137,7 +137,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
// Some needed configuration for brokers, producers, and consumers
this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1")
this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3")
- this.serverConfig.setProperty(KafkaConfig.MinInSyncReplicasProp, "3")
+ this.serverConfig.setProperty(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3")
this.serverConfig.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "3")
this.serverConfig.setProperty(KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, "1500")
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "group")
diff --git a/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala
index ba7f5f68cc503..17f12ab37a387 100644
--- a/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala
+++ b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala
@@ -18,11 +18,11 @@ package kafka.api
import java.util.Collections
import java.util.concurrent.TimeUnit
-import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.{BeforeEach, TestInfo}
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue}
import org.junit.jupiter.params.ParameterizedTest
@@ -38,7 +38,7 @@ class LogAppendTimeTest extends IntegrationTestHarness {
val brokerCount: Int = 2
// This will be used for the offsets topic as well
- serverConfig.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.name)
+ serverConfig.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.name)
serverConfig.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "2")
private val topic = "topic"
diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala
index 88fca09b11f14..17bd71a156f5c 100644
--- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala
+++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala
@@ -13,7 +13,7 @@
package kafka.api
import java.util.{Locale, Properties}
-import kafka.server.{KafkaConfig, KafkaServer}
+import kafka.server.KafkaServer
import kafka.utils.{JaasTestUtils, TestUtils}
import com.yammer.metrics.core.{Gauge, Histogram, Meter}
import org.apache.kafka.clients.consumer.Consumer
@@ -25,7 +25,7 @@ import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.security.authenticator.TestJaasConfig
import org.apache.kafka.server.config.ZkConfigs
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteStorageMetrics}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
@@ -46,7 +46,7 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {
private val kafkaServerJaasEntryName =
s"${listenerName.value.toLowerCase(Locale.ROOT)}.${JaasTestUtils.KafkaServerContextName}"
this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "false")
- this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false")
+ this.serverConfig.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
this.serverConfig.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "2.8")
this.producerConfig.setProperty(ProducerConfig.LINGER_MS_CONFIG, "10")
// intentionally slow message down conversion via gzip compression to ensure we can measure the time it takes
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index 3cbdf84a1e64c..4a39cf6c44c26 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -45,7 +45,7 @@ import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollectio
import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry
-import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs}
+import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs, ZkConfigs}
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo}
@@ -2222,7 +2222,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertEquals(compressionType, logConfig.originals.get(TopicConfig.COMPRESSION_TYPE_CONFIG))
assertNull(logConfig.originals.get(TopicConfig.RETENTION_BYTES_CONFIG))
- assertEquals(LogConfig.DEFAULT_RETENTION_BYTES, logConfig.retentionSize)
+ assertEquals(ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, logConfig.retentionSize)
}
client = Admin.create(createConfig)
@@ -2528,7 +2528,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
client = Admin.create(super.createConfig)
val newLogRetentionProperties = new Properties
- newLogRetentionProperties.put(KafkaConfig.LogRetentionTimeMillisProp, "10800000")
+ newLogRetentionProperties.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "10800000")
TestUtils.incrementalAlterConfigs(null, client, newLogRetentionProperties, perBrokerConfig = false)
.all().get(15, TimeUnit.SECONDS)
@@ -2555,8 +2555,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
waitTimeMs = 60000L)
waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault(
- KafkaConfig.LogRetentionTimeMillisProp, "").toString.equals("10800000")),
- s"Timed out waiting for change to ${KafkaConfig.LogRetentionTimeMillisProp}",
+ ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "").toString.equals("10800000")),
+ s"Timed out waiting for change to ${ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG}",
waitTimeMs = 60000L)
val newTopics = Seq(new NewTopic("foo", Map((0: Integer) -> Seq[Integer](1, 2).asJava,
diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
index 10b6095484125..cf607489a001d 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
@@ -28,7 +28,7 @@ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -43,7 +43,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
val numServers = 2
val overridingProps = new Properties()
- overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString)
overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, replicaFetchMaxPartitionBytes.toString)
overridingProps.put(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_DOC, replicaFetchMaxResponseBytes.toString)
diff --git a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala
index db7ca5ad7e64b..5aa00dda43c23 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala
@@ -31,7 +31,7 @@ import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.test.{TestUtils => JTestUtils}
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
@@ -226,7 +226,7 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness {
private def serverProps(): Properties = {
val serverProps = new Properties()
- serverProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ serverProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
// Set a smaller value for the number of partitions for the __consumer_offsets topic
// so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long.
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala
index 6a044bcee21d1..365a872d01b9b 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala
@@ -16,12 +16,11 @@
*/
package kafka.api
-import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import org.apache.kafka.clients.admin.NewPartitionReassignment
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
@@ -36,7 +35,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
val producerCount: Int = 1
val brokerCount: Int = 2
- serverConfig.put(KafkaConfig.NumPartitionsProp, 2.toString)
+ serverConfig.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 2.toString)
serverConfig.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, 2.toString)
serverConfig.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString)
diff --git a/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala
index c7d2855e17266..a8d2431f80bb3 100644
--- a/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala
@@ -21,9 +21,9 @@ import java.util.Properties
import kafka.admin.{RackAwareMode, RackAwareTest}
import kafka.integration.KafkaServerTestHarness
import kafka.server.KafkaConfig
-import org.apache.kafka.server.config.ReplicationConfigs
import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.ProducerRecord
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import scala.collection.Map
@@ -33,7 +33,7 @@ class RackAwareAutoTopicCreationTest extends KafkaServerTestHarness with RackAwa
val numPartitions = 8
val replicationFactor = 2
val overridingProps = new Properties()
- overridingProps.put(KafkaConfig.NumPartitionsProp, numPartitions.toString)
+ overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numPartitions.toString)
overridingProps.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, replicationFactor.toString)
def generateConfigs =
diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala
index 67656d27275ec..53034f84fd25e 100644
--- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala
@@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.util.ShutdownableThread
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
@@ -45,7 +45,7 @@ class TransactionsBounceTest extends IntegrationTestHarness {
private val inputTopic = "input-topic"
val overridingProps = new Properties()
- overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString)
// Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic)
// so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long
@@ -54,7 +54,7 @@ class TransactionsBounceTest extends IntegrationTestHarness {
overridingProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString)
overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 3.toString)
- overridingProps.put(KafkaConfig.MinInSyncReplicasProp, 2.toString)
+ overridingProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 2.toString)
overridingProps.put(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "10") // set small enough session timeout
overridingProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0")
overridingProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString)
@@ -187,7 +187,7 @@ class TransactionsBounceTest extends IntegrationTestHarness {
private def createTopics() = {
val topicConfig = new Properties()
- topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 2.toString)
+ topicConfig.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 2.toString)
createTopic(inputTopic, numPartitions, 3, topicConfig)
createTopic(outputTopic, numPartitions, 3, topicConfig)
}
diff --git a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala
index e639434f36dab..ee39a764f2ece 100644
--- a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala
@@ -29,7 +29,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException}
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -201,7 +201,7 @@ class TransactionsExpirationTest extends KafkaServerTestHarness {
private def serverProps(): Properties = {
val serverProps = new Properties()
- serverProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ serverProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
// Set a smaller value for the number of partitions for the __consumer_offsets topic
// so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long.
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
index 5e4deaaa834de..2417667237f09 100644
--- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
+++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
@@ -31,7 +31,7 @@ import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFe
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -61,7 +61,7 @@ class TransactionsTest extends IntegrationTestHarness {
def overridingProps(): Properties = {
val props = new Properties()
- props.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ props.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
// Set a smaller value for the number of partitions for the __consumer_offsets topic + // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long
props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString)
@@ -92,7 +92,7 @@ class TransactionsTest extends IntegrationTestHarness {
def topicConfig(): Properties = {
val topicConfig = new Properties()
- topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 2.toString)
+ topicConfig.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 2.toString)
topicConfig
}
diff --git a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala
index 1865c0a8230e5..58455223f2caf 100644
--- a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala
+++ b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala
@@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -58,7 +58,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness {
override def setUp(testInfo: TestInfo): Unit = {
super.setUp(testInfo)
val topicConfig = new Properties()
- topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 1.toString)
+ topicConfig.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 1.toString)
createTopic(topic1, numPartitions, numBrokers, topicConfig)
createTopic(topic2, numPartitions, numBrokers, topicConfig)
@@ -105,7 +105,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness {
private def serverProps() = {
val serverProps = new Properties()
- serverProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ serverProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 1.toString)
serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString)
diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
index 0100ff055cf42..e881d427c73a1 100644
--- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
@@ -62,7 +62,7 @@ import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializ
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ZkConfigs}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.record.BrokerCompressionType
import org.apache.kafka.server.util.ShutdownableThread
@@ -135,11 +135,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
props.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "requested")
props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN")
props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(","))
- props.put(KafkaConfig.LogSegmentBytesProp, "2000") // low value to test log rolling on config update
+ props.put(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "2000") // low value to test log rolling on config update
props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") // greater than one to test reducing threads
props.put(KafkaConfig.PasswordEncoderSecretProp, "dynamic-config-secret")
- props.put(KafkaConfig.LogRetentionTimeMillisProp, 1680000000.toString)
- props.put(KafkaConfig.LogRetentionTimeHoursProp, 168.toString)
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, 1680000000.toString)
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, 168.toString)
props ++= sslProperties1
props ++= securityProps(sslProperties1, KEYSTORE_PROPS, listenerPrefix(SecureInternal))
@@ -253,18 +253,18 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify a few log configs with and without synonyms
val expectedProps = new Properties
- expectedProps.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1680000000")
- expectedProps.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "168")
- expectedProps.setProperty(KafkaConfig.LogRollTimeHoursProp, "168")
+ expectedProps.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1680000000")
+ expectedProps.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "168")
+ expectedProps.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "168")
expectedProps.setProperty(CleanerConfig.LOG_CLEANER_THREADS_PROP, "1")
- val logRetentionMs = configEntry(configDesc, KafkaConfig.LogRetentionTimeMillisProp)
- verifyConfig(KafkaConfig.LogRetentionTimeMillisProp, logRetentionMs,
+ val logRetentionMs = configEntry(configDesc, ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)
+ verifyConfig(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, logRetentionMs,
isSensitive = false, isReadOnly = false, expectedProps)
- val logRetentionHours = configEntry(configDesc, KafkaConfig.LogRetentionTimeHoursProp)
- verifyConfig(KafkaConfig.LogRetentionTimeHoursProp, logRetentionHours,
+ val logRetentionHours = configEntry(configDesc, ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG)
+ verifyConfig(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, logRetentionHours,
isSensitive = false, isReadOnly = true, expectedProps)
- val logRollHours = configEntry(configDesc, KafkaConfig.LogRollTimeHoursProp)
- verifyConfig(KafkaConfig.LogRollTimeHoursProp, logRollHours,
+ val logRollHours = configEntry(configDesc, ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG)
+ verifyConfig(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, logRollHours,
isSensitive = false, isReadOnly = true, expectedProps)
val logCleanerThreads = configEntry(configDesc, CleanerConfig.LOG_CLEANER_THREADS_PROP)
verifyConfig(CleanerConfig.LOG_CLEANER_THREADS_PROP, logCleanerThreads,
@@ -272,14 +272,14 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
def synonymsList(configEntry: ConfigEntry): List[(String, ConfigSource)] =
configEntry.synonyms.asScala.map(s => (s.name, s.source)).toList
- assertEquals(List((KafkaConfig.LogRetentionTimeMillisProp, ConfigSource.STATIC_BROKER_CONFIG),
- (KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.STATIC_BROKER_CONFIG),
- (KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.DEFAULT_CONFIG)),
+ assertEquals(List((ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, ConfigSource.STATIC_BROKER_CONFIG),
+ (ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.STATIC_BROKER_CONFIG),
+ (ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.DEFAULT_CONFIG)),
synonymsList(logRetentionMs))
- assertEquals(List((KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.STATIC_BROKER_CONFIG),
- (KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.DEFAULT_CONFIG)),
+ assertEquals(List((ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.STATIC_BROKER_CONFIG),
+ (ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.DEFAULT_CONFIG)),
synonymsList(logRetentionHours))
- assertEquals(List((KafkaConfig.LogRollTimeHoursProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logRollHours))
+ assertEquals(List((ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, ConfigSource.DEFAULT_CONFIG)), synonymsList(logRollHours))
assertEquals(List((CleanerConfig.LOG_CLEANER_THREADS_PROP, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads))
}
@@ -581,7 +581,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
def testConsecutiveConfigChange(quorum: String): Unit = {
val topic2 = "testtopic2"
val topicProps = new Properties
- topicProps.put(KafkaConfig.MinInSyncReplicasProp, "2")
+ topicProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2")
TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, controllerServers, numPartitions = 1, replicationFactor = numServers, topicConfig = topicProps)
def getLogOrThrow(tp: TopicPartition): UnifiedLog = {
@@ -593,13 +593,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
var log = getLogOrThrow(new TopicPartition(topic2, 0))
- assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp))
- assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString)
+ assertTrue(log.config.overriddenConfigs.contains(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG))
+ assertEquals("2", log.config.originals().get(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG).toString)
val props = new Properties
- props.put(KafkaConfig.MinInSyncReplicasProp, "3")
+ props.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3")
// Make a broker-default config
- reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.MinInSyncReplicasProp, "3"))
+ reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3"))
// Verify that all broker defaults have been updated again
servers.foreach { server =>
props.forEach { (k, v) =>
@@ -608,16 +608,16 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
log = getLogOrThrow(new TopicPartition(topic2, 0))
- assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp))
- assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config survives
+ assertTrue(log.config.overriddenConfigs.contains(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG))
+ assertEquals("2", log.config.originals().get(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG).toString) // Verify topic-level config survives
// Make a second broker-default change
props.clear()
- props.put(KafkaConfig.LogRetentionTimeMillisProp, "604800000")
- reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogRetentionTimeMillisProp, "604800000"))
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "604800000")
+ reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "604800000"))
log = getLogOrThrow(new TopicPartition(topic2, 0))
- assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp))
- assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config still survives
+ assertTrue(log.config.overriddenConfigs.contains(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG))
+ assertEquals("2", log.config.originals().get(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG).toString) // Verify topic-level config still survives
}
@Test
@@ -627,31 +627,31 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val (producerThread, consumerThread) = startProduceConsume(retries = 0)
val props = new Properties
- props.put(KafkaConfig.LogSegmentBytesProp, "4000")
- props.put(KafkaConfig.LogRollTimeMillisProp, TimeUnit.HOURS.toMillis(2).toString)
- props.put(KafkaConfig.LogRollTimeJitterMillisProp, TimeUnit.HOURS.toMillis(1).toString)
- props.put(KafkaConfig.LogIndexSizeMaxBytesProp, "100000")
- props.put(KafkaConfig.LogFlushIntervalMessagesProp, "1000")
- props.put(KafkaConfig.LogFlushIntervalMsProp, "60000")
- props.put(KafkaConfig.LogRetentionBytesProp, "10000000")
- props.put(KafkaConfig.LogRetentionTimeMillisProp, TimeUnit.DAYS.toMillis(1).toString)
+ props.put(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "4000")
+ props.put(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, TimeUnit.HOURS.toMillis(2).toString)
+ props.put(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, TimeUnit.HOURS.toMillis(1).toString)
+ props.put(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, "100000")
+ props.put(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG, "1000")
+ props.put(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, "60000")
+ props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "10000000")
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, TimeUnit.DAYS.toMillis(1).toString)
props.put(KafkaConfig.MessageMaxBytesProp, "100000")
- props.put(KafkaConfig.LogIndexIntervalBytesProp, "10000")
+ props.put(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG, "10000")
props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString)
props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000")
- props.put(KafkaConfig.LogDeleteDelayMsProp, "60000")
+ props.put(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "60000")
props.put(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP, "0.3")
- props.put(KafkaConfig.LogCleanupPolicyProp, "delete")
+ props.put(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG, "delete")
props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false")
- props.put(KafkaConfig.MinInSyncReplicasProp, "2")
+ props.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2")
props.put(KafkaConfig.CompressionTypeProp, "gzip")
- props.put(KafkaConfig.LogPreAllocateProp, true.toString)
- props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.toString)
- props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000")
- props.put(KafkaConfig.LogMessageTimestampBeforeMaxMsProp, "1000")
- props.put(KafkaConfig.LogMessageTimestampAfterMaxMsProp, "1000")
- props.put(KafkaConfig.LogMessageDownConversionEnableProp, "false")
- reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogSegmentBytesProp, "4000"))
+ props.put(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG, true.toString)
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.toString)
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000")
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, "1000")
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, "1000")
+ props.put(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "false")
+ reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "4000"))
// Verify that all broker defaults have been updated
servers.foreach { server =>
@@ -669,7 +669,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
TestUtils.waitUntilTrue(() => log.config.segmentSize == 4000, "Existing topic config using defaults not updated")
props.asScala.foreach { case (k, v) =>
val logConfigName = DynamicLogConfig.KafkaConfigToLogConfigName(k)
- val expectedValue = if (k == KafkaConfig.LogCleanupPolicyProp) s"[$v]" else v
+ val expectedValue = if (k == ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG) s"[$v]" else v
assertEquals(expectedValue, log.config.originals.get(logConfigName).toString,
s"Not reconfigured $logConfigName for existing log")
}
@@ -686,19 +686,19 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify that we can alter subset of log configs
props.clear()
- props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.CREATE_TIME.toString)
- 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))
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.CREATE_TIME.toString)
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000")
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, "1000")
+ props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, "1000")
+ reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.CREATE_TIME.toString))
consumerThread.waitForMatchingRecords(record => record.timestampType == TimestampType.CREATE_TIME)
// Verify that invalid configs are not applied
val invalidProps = Map(
- KafkaConfig.LogMessageTimestampDifferenceMaxMsProp -> "abc", // Invalid type
- KafkaConfig.LogMessageTimestampBeforeMaxMsProp -> "abc", // Invalid type
- KafkaConfig.LogMessageTimestampAfterMaxMsProp -> "abc", // Invalid type
- KafkaConfig.LogMessageTimestampTypeProp -> "invalid", // Invalid value
- KafkaConfig.LogRollTimeMillisProp -> "0" // Fails KafkaConfig validation
+ ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG -> "abc", // Invalid type
+ ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG -> "abc", // Invalid type
+ ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG -> "abc", // Invalid type
+ ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG -> "invalid", // Invalid value
+ ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG -> "0" // Fails KafkaConfig validation
)
invalidProps.foreach { case (k, v) =>
val newProps = new Properties
@@ -710,14 +710,14 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify that even though broker defaults can be defined at default cluster level for consistent
// configuration across brokers, they can also be defined at per-broker level for testing
props.clear()
- props.put(KafkaConfig.LogIndexSizeMaxBytesProp, "500000")
- props.put(KafkaConfig.LogRetentionTimeMillisProp, TimeUnit.DAYS.toMillis(2).toString)
+ props.put(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, "500000")
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, TimeUnit.DAYS.toMillis(2).toString)
alterConfigsOnServer(servers.head, props)
- assertEquals(500000, servers.head.config.values.get(KafkaConfig.LogIndexSizeMaxBytesProp))
- assertEquals(TimeUnit.DAYS.toMillis(2), servers.head.config.values.get(KafkaConfig.LogRetentionTimeMillisProp))
+ assertEquals(500000, servers.head.config.values.get(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG))
+ assertEquals(TimeUnit.DAYS.toMillis(2), servers.head.config.values.get(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG))
servers.tail.foreach { server =>
- assertEquals(LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, server.config.values.get(KafkaConfig.LogIndexSizeMaxBytesProp))
- assertEquals(1680000000L, server.config.values.get(KafkaConfig.LogRetentionTimeMillisProp))
+ assertEquals(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT, server.config.values.get(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG))
+ assertEquals(1680000000L, server.config.values.get(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG))
}
// Verify that produce/consume worked throughout this test without any retries in producer
@@ -726,17 +726,17 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify that configuration at both per-broker level and default cluster level could be deleted and
// the default value should be restored
props.clear()
- props.put(KafkaConfig.LogRetentionTimeMillisProp, "")
- props.put(KafkaConfig.LogIndexSizeMaxBytesProp, "")
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "")
+ props.put(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, "")
TestUtils.incrementalAlterConfigs(servers.take(1), adminClients.head, props, perBrokerConfig = true, opType = OpType.DELETE).all.get
TestUtils.incrementalAlterConfigs(servers, adminClients.head, props, perBrokerConfig = false, opType = OpType.DELETE).all.get
servers.foreach { server =>
- waitForConfigOnServer(server, KafkaConfig.LogRetentionTimeMillisProp, 1680000000.toString)
+ waitForConfigOnServer(server, ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, 1680000000.toString)
}
servers.foreach { server =>
val log = server.logManager.getLog(new TopicPartition(topic, 0)).getOrElse(throw new IllegalStateException("Log not found"))
// Verify default values for these two configurations are restored on all brokers
- TestUtils.waitUntilTrue(() => log.config.maxIndexSize == LogConfig.DEFAULT_SEGMENT_INDEX_BYTES && log.config.retentionMs == 1680000000L,
+ TestUtils.waitUntilTrue(() => log.config.maxIndexSize == ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT && log.config.retentionMs == 1680000000L,
"Existing topic config using defaults not updated")
}
}
@@ -876,7 +876,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
fetcherThreadPrefix, mayReceiveDuplicates = false)
verifyThreadPoolResize(KafkaConfig.BackgroundThreadsProp, config.backgroundThreads,
"kafka-scheduler-", mayReceiveDuplicates = false)
- verifyThreadPoolResize(KafkaConfig.NumRecoveryThreadsPerDataDirProp, config.numRecoveryThreadsPerDataDir,
+ verifyThreadPoolResize(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, config.numRecoveryThreadsPerDataDir,
"", mayReceiveDuplicates = false)
verifyThreadPoolResize(KafkaConfig.NumNetworkThreadsProp, config.numNetworkThreads,
networkThreadPrefix, mayReceiveDuplicates = true)
diff --git a/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala
index ce4cccc076ffc..c9f3c46f0d8c2 100644
--- a/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala
@@ -25,6 +25,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.FetchResponse
import org.apache.kafka.common.serialization.ByteArrayDeserializer
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{Disabled, Timeout}
import org.junit.jupiter.params.ParameterizedTest
@@ -45,7 +46,7 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
def overridingProps: Properties = {
val props = new Properties
- props.put(KafkaConfig.NumPartitionsProp, numParts.toString)
+ props.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numParts.toString)
props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, numNodes.toString)
props
}
diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
index 508a82a511c4b..4b51dd3f8ad85 100644
--- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
+++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
@@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.raft._
import org.apache.kafka.raft.internals.BatchBuilder
import org.apache.kafka.server.common.serialization.RecordSerde
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.MockTime
import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason}
@@ -842,7 +843,7 @@ final class KafkaMetadataLogTest {
retentionMillis = 60 * 1000,
maxBatchSizeInBytes = 512,
maxFetchSizeInBytes = DefaultMetadataLogConfig.maxFetchSizeInBytes,
- fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS,
+ fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
nodeId = 1
)
config.copy()
@@ -1024,7 +1025,7 @@ object KafkaMetadataLogTest {
retentionMillis = 60 * 1000,
maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES,
maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES,
- fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS,
+ fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
nodeId = 1
)
diff --git a/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala
index eecc616dc88c5..7f8db18166c65 100644
--- a/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala
@@ -26,6 +26,7 @@ import kafka.server.KafkaConfig
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.log4j.Logger
import org.junit.jupiter.api.{AfterEach, Test}
import org.junit.jupiter.api.Assertions._
@@ -38,7 +39,7 @@ class ControllerFailoverTest extends KafkaServerTestHarness with Logging {
val topic = "topic1"
val overridingProps = new Properties()
val metrics = new Metrics()
- overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString)
+ overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numParts.toString)
override def generateConfigs = TestUtils.createBrokerConfigs(numNodes, zkConnect)
.map(KafkaConfig.fromProps(_, overridingProps))
diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala
index cb12735535742..ac7b62e179433 100644
--- a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala
@@ -24,7 +24,7 @@ import kafka.utils.{Logging, TestUtils}
import scala.jdk.CollectionConverters._
import org.junit.jupiter.api.{BeforeEach, TestInfo}
import com.yammer.metrics.core.Gauge
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
@@ -40,7 +40,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with
private val overridingProps = new Properties
overridingProps.put(KafkaConfig.DeleteTopicEnableProp, "true")
- overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, "false")
+ overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
// speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation
// But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment
overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, "4000")
diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala
index 603f727516277..572df4d0b1327 100644
--- a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala
@@ -22,12 +22,13 @@ import scala.collection.Seq
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
class MinIsrConfigTest extends KafkaServerTestHarness {
val overridingProps = new Properties()
- overridingProps.put(KafkaConfig.MinInSyncReplicasProp, "5")
+ overridingProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "5")
def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps(_, overridingProps))
@ParameterizedTest
diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
index 697d0e1f2694c..e0fc02a5bab91 100644
--- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
@@ -27,7 +27,7 @@ import org.junit.jupiter.api.Test
import java.util.{Collections, Properties}
import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1
-import org.apache.kafka.server.config.KafkaSecurityConfigs
+import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator}
import org.junit.jupiter.params.ParameterizedTest
@@ -48,7 +48,7 @@ class LogConfigTest {
@Test
def ensureNoStaticInitializationOrderDependency(): Unit = {
// Access any KafkaConfig val to load KafkaConfig object before LogConfig.
- assertNotNull(KafkaConfig.LogRetentionTimeMillisProp)
+ assertNotNull(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)
assertTrue(LogConfig.configNames.asScala
.filter(config => !LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS.contains(config))
.forall { config =>
@@ -64,10 +64,10 @@ class LogConfigTest {
val millisInDay = 24L * millisInHour
val bytesInGB: Long = 1024 * 1024 * 1024
val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
- kafkaProps.put(KafkaConfig.LogRollTimeHoursProp, "2")
- kafkaProps.put(KafkaConfig.LogRollTimeJitterHoursProp, "2")
- kafkaProps.put(KafkaConfig.LogRetentionTimeHoursProp, "960") // 40 days
- kafkaProps.put(KafkaConfig.LogMessageFormatVersionProp, "0.11.0")
+ kafkaProps.put(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "2")
+ kafkaProps.put(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG, "2")
+ kafkaProps.put(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "960") // 40 days
+ kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "2592000000") // 30 days
kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "4294967296") // 4 GB
@@ -173,7 +173,7 @@ class LogConfigTest {
val deleteDelayKey = configDef.configKeys.get(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG)
val deleteDelayServerDefault = configDef.getConfigValue(deleteDelayKey, LogConfig.SERVER_DEFAULT_HEADER_NAME)
- assertEquals(KafkaConfig.LogDeleteDelayMsProp, deleteDelayServerDefault)
+ assertEquals(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, deleteDelayServerDefault)
val keyWithNoServerMapping = configDef.configKeys.get(configNameWithNoServerMapping)
val nullServerDefault = configDef.getConfigValue(keyWithNoServerMapping, LogConfig.SERVER_DEFAULT_HEADER_NAME)
@@ -184,8 +184,8 @@ class LogConfigTest {
def testOverriddenConfigsAsLoggableString(): Unit = {
val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
kafkaProps.put("unknown.broker.password.config", "aaaaa")
+ kafkaProps.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "50")
kafkaProps.put(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, "somekeypassword")
- kafkaProps.put(KafkaConfig.LogRetentionBytesProp, "50")
val kafkaConfig = KafkaConfig.fromProps(kafkaProps)
val topicOverrides = new Properties
// Only set as a topic config
@@ -238,7 +238,7 @@ class LogConfigTest {
// Local retention defaults are derived from retention properties which can be default or custom.
assertEquals(LogConfig.DEFAULT_RETENTION_MS, logConfig.localRetentionMs)
- assertEquals(LogConfig.DEFAULT_RETENTION_BYTES, logConfig.localRetentionBytes)
+ assertEquals(ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, logConfig.localRetentionBytes)
}
@Test
@@ -342,7 +342,7 @@ class LogConfigTest {
def testTopicCreationWithInvalidRetentionTime(sysRemoteStorageEnabled: Boolean): Unit = {
val kafkaProps = TestUtils.createDummyBrokerConfig()
kafkaProps.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
- kafkaProps.put(KafkaConfig.LogRetentionTimeMillisProp, "1000")
+ kafkaProps.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1000")
kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "900")
val kafkaConfig = KafkaConfig.fromProps(kafkaProps)
@@ -364,7 +364,7 @@ class LogConfigTest {
def testTopicCreationWithInvalidRetentionSize(sysRemoteStorageEnabled: Boolean): Unit = {
val props = TestUtils.createDummyBrokerConfig()
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
- props.put(KafkaConfig.LogRetentionBytesProp, "1024")
+ props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "1024")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "512")
val kafkaConfig = KafkaConfig.fromProps(props)
@@ -386,7 +386,7 @@ class LogConfigTest {
def testValidateBrokerLogConfigs(sysRemoteStorageEnabled: Boolean): Unit = {
val props = TestUtils.createDummyBrokerConfig()
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
- props.put(KafkaConfig.LogRetentionBytesProp, "1024")
+ props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "1024")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "2048")
val kafkaConfig = KafkaConfig.fromProps(props)
@@ -399,8 +399,8 @@ class LogConfigTest {
}
}
- /* 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
+ /* Verify that when the deprecated config LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG has non default value the new configs
+ * LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG and LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG 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")
@@ -408,9 +408,9 @@ class LogConfigTest {
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)
+ kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.MaxValue.toString)
+ kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.MaxValue.toString)
+ kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, oneDayInMillis.toString)
val logProps = KafkaConfig.fromProps(kafkaProps).extractLogConfigMap
assertEquals(oneDayInMillis, logProps.get(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG))
diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
index e193baf92e523..e583f8cf6cbf3 100644
--- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
@@ -32,6 +32,7 @@ import java.nio.file.Files
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile
import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, FetchDataInfo, FetchIsolation, LazyIndex, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetsListener, LogSegment, ProducerStateManager, ProducerStateManagerConfig, TransactionIndex}
@@ -58,14 +59,14 @@ object LogTestUtils {
segmentBytes: Int = LogConfig.DEFAULT_SEGMENT_BYTES,
retentionMs: Long = LogConfig.DEFAULT_RETENTION_MS,
localRetentionMs: Long = LogConfig.DEFAULT_LOCAL_RETENTION_MS,
- retentionBytes: Long = LogConfig.DEFAULT_RETENTION_BYTES,
+ retentionBytes: Long = ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT,
localRetentionBytes: Long = LogConfig.DEFAULT_LOCAL_RETENTION_BYTES,
segmentJitterMs: Long = LogConfig.DEFAULT_SEGMENT_JITTER_MS,
- cleanupPolicy: String = LogConfig.DEFAULT_CLEANUP_POLICY,
+ cleanupPolicy: String = ServerLogConfigs.LOG_CLEANUP_POLICY_DEFAULT,
maxMessageBytes: Int = LogConfig.DEFAULT_MAX_MESSAGE_BYTES,
- indexIntervalBytes: Int = LogConfig.DEFAULT_INDEX_INTERVAL_BYTES,
- segmentIndexBytes: Int = LogConfig.DEFAULT_SEGMENT_INDEX_BYTES,
- fileDeleteDelayMs: Long = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS,
+ indexIntervalBytes: Int = ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT,
+ segmentIndexBytes: Int = ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT,
+ fileDeleteDelayMs: Long = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
remoteLogStorageEnable: Boolean = LogConfig.DEFAULT_REMOTE_STORAGE_ENABLE): LogConfig = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long)
diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
index 5935346a0252e..4f5c853a8cae2 100644
--- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
+++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
@@ -33,6 +33,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.metrics.JmxReporter
import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.migration.ZkMigrationState
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.Timeout
@@ -46,7 +47,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
val requiredKafkaServerPrefix = "kafka.server:type=KafkaServer,name"
val overridingProps = new Properties
- overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString)
+ overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numParts.toString)
overridingProps.put(JmxReporter.EXCLUDE_CONFIG, s"$requiredKafkaServerPrefix=ClusterId")
def generateConfigs: Seq[KafkaConfig] =
diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala
index b682a34edb63e..c082c3dddc145 100644
--- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala
@@ -30,7 +30,7 @@ import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.utils.Time
import org.apache.kafka.raft.RaftConfig
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.config.ZkConfigs
import org.junit.jupiter.api.Assertions._
@@ -50,7 +50,7 @@ class RaftManagerTest {
): KafkaConfig = {
val props = new Properties
logDir.foreach { value =>
- props.setProperty(KafkaConfig.LogDirProp, value.toString)
+ props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, value.toString)
}
if (migrationEnabled) {
metadataDir.foreach { value =>
@@ -74,7 +74,7 @@ class RaftManagerTest {
): KafkaConfig = {
val props = new Properties
logDir.foreach { value =>
- props.setProperty(KafkaConfig.LogDirProp, value.toString)
+ props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, value.toString)
}
metadataDir.foreach { value =>
props.setProperty(KafkaConfig.MetadataLogDirProp, value.toString)
diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala
index 627a939ddcf77..296ee7cc20eb5 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala
@@ -26,6 +26,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData
import org.apache.kafka.common.message.CreateTopicsRequestData._
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests._
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
import scala.jdk.CollectionConverters._
@@ -33,7 +34,7 @@ import scala.jdk.CollectionConverters._
abstract class AbstractCreateTopicsRequestTest extends BaseRequestTest {
override def brokerPropertyOverrides(properties: Properties): Unit =
- properties.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ properties.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
def topicsReq(topics: Seq[CreatableTopic],
timeout: Integer = 10000,
diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala
index 26876aa7d7062..86433a50d9fca 100644
--- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala
@@ -30,6 +30,7 @@ import org.apache.kafka.common.message.{FindCoordinatorRequestData, InitProducer
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType
import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, FindCoordinatorRequest, FindCoordinatorResponse, InitProducerIdRequest, InitProducerIdResponse}
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -43,7 +44,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
val numPartitions = 1
override def brokerPropertyOverrides(properties: Properties): Unit = {
- properties.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString)
+ properties.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
}
@BeforeEach
diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
index a36813986e8d9..9f0a4c1022dc4 100644
--- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
@@ -25,6 +25,7 @@ import org.apache.kafka.common.message.{BrokerHeartbeatResponseData, BrokerRegis
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
import org.apache.kafka.metadata.BrokerState
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{Test, Timeout}
@@ -35,7 +36,7 @@ import scala.jdk.CollectionConverters._
class BrokerLifecycleManagerTest {
def configProperties = {
val properties = new Properties()
- properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo")
+ properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "broker")
properties.setProperty(KafkaConfig.NodeIdProp, "1")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9093")
diff --git a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala
index 694b13d041a70..3776d0c565fa8 100644
--- a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala
@@ -28,6 +28,7 @@ import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.metadata.{ListenerInfo, RecordTestUtils, VersionRange}
import org.apache.kafka.raft.LeaderAndEpoch
import org.apache.kafka.server.common.MetadataVersion
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.{Test, Timeout}
@@ -45,7 +46,7 @@ class ControllerRegistrationManagerTest {
private def configProperties = {
val properties = new Properties()
- properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo")
+ properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller")
properties.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, s"CONTROLLER:PLAINTEXT")
properties.setProperty(KafkaConfig.ListenersProp, s"CONTROLLER://localhost:8001")
diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala
index c51cd6a5a7b35..05f248e5ca7af 100644
--- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala
+++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala
@@ -23,6 +23,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.PolicyViolationException
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG
import org.apache.kafka.server.policy.CreateTopicPolicy
import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata
import org.junit.jupiter.params.ParameterizedTest
@@ -35,12 +36,12 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest
override def brokerPropertyOverrides(properties: Properties): Unit = {
super.brokerPropertyOverrides(properties)
- properties.put(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[Policy].getName)
+ properties.put(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[Policy].getName)
}
override def kraftControllerConfigs(): Seq[Properties] = {
val properties = new Properties()
- properties.put(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[Policy].getName)
+ properties.put(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[Policy].getName)
Seq(properties)
}
diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
index 4a094124e1d94..51a5bce13ca7a 100755
--- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
@@ -38,7 +38,7 @@ import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs, ZkConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.KafkaScheduler
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig}
import org.apache.kafka.test.MockMetricsReporter
import org.junit.jupiter.api.Assertions._
@@ -135,7 +135,7 @@ class DynamicBrokerConfigTest {
origProps.put(KafkaConfig.NumIoThreadsProp, "4")
origProps.put(KafkaConfig.NumNetworkThreadsProp, "2")
origProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1")
- origProps.put(KafkaConfig.NumRecoveryThreadsPerDataDirProp, "1")
+ origProps.put(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, "1")
origProps.put(KafkaConfig.BackgroundThreadsProp, "3")
val config = KafkaConfig(origProps)
@@ -181,7 +181,7 @@ class DynamicBrokerConfigTest {
assertEquals(2, config.numReplicaFetchers)
Mockito.verify(replicaManagerMock).resizeFetcherThreadPool(newSize = 2)
- props.put(KafkaConfig.NumRecoveryThreadsPerDataDirProp, "2")
+ props.put(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, "2")
config.dynamicConfig.updateDefaultConfig(props)
assertEquals(2, config.numRecoveryThreadsPerDataDir)
Mockito.verify(logManagerMock).resizeRecoveryThreadPool(newSize = 2)
@@ -219,7 +219,7 @@ class DynamicBrokerConfigTest {
val invalidProps = Map(CleanerConfig.LOG_CLEANER_THREADS_PROP -> "invalid")
verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, invalidProps)
- val excludedTopicConfig = Map(KafkaConfig.LogMessageFormatVersionProp -> "0.10.2")
+ val excludedTopicConfig = Map(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG -> "0.10.2")
verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, excludedTopicConfig)
}
@@ -604,8 +604,8 @@ class DynamicBrokerConfigTest {
DynamicBrokerConfig.brokerConfigSynonyms("listener.name.sasl_ssl.plain.sasl.jaas.config", matchListenerOverride = true))
assertEquals(List("some.config"),
DynamicBrokerConfig.brokerConfigSynonyms("some.config", matchListenerOverride = true))
- assertEquals(List(KafkaConfig.LogRollTimeMillisProp, KafkaConfig.LogRollTimeHoursProp),
- DynamicBrokerConfig.brokerConfigSynonyms(KafkaConfig.LogRollTimeMillisProp, matchListenerOverride = true))
+ assertEquals(List(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG),
+ DynamicBrokerConfig.brokerConfigSynonyms(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, matchListenerOverride = true))
}
@Test
@@ -721,7 +721,7 @@ class DynamicBrokerConfigTest {
@Test
def testDynamicLogLocalRetentionMsConfig(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.put(KafkaConfig.LogRetentionTimeMillisProp, "2592000000")
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "2592000000")
val config = KafkaConfig(props)
val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer]))
config.dynamicConfig.initialize(None, None)
@@ -744,7 +744,7 @@ class DynamicBrokerConfigTest {
@Test
def testDynamicLogLocalRetentionSizeConfig(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.put(KafkaConfig.LogRetentionBytesProp, "4294967296")
+ props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "4294967296")
val config = KafkaConfig(props)
val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer]))
config.dynamicConfig.initialize(None, None)
@@ -820,8 +820,8 @@ class DynamicBrokerConfigTest {
logLocalRetentionBytes: Long,
retentionBytes: Long): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.put(KafkaConfig.LogRetentionTimeMillisProp, retentionMs.toString)
- props.put(KafkaConfig.LogRetentionBytesProp, retentionBytes.toString)
+ props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, retentionMs.toString)
+ props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, retentionBytes.toString)
val config = KafkaConfig(props)
val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer]))
config.dynamicConfig.initialize(None, None)
diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
index abe81d9652717..921ead3de81d3 100644
--- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
@@ -44,7 +44,7 @@ import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity}
import org.apache.kafka.common.record.{CompressionType, RecordVersion}
import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1
-import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals}
+import org.apache.kafka.server.config.{ConfigType, ServerLogConfigs, ZooKeeperInternals}
import org.apache.kafka.storage.internals.log.LogConfig
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{Test, Timeout}
@@ -87,7 +87,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
val op = new AlterConfigOp(new ConfigEntry(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, newVal.toString()),
SET)
val resource2 = new ConfigResource(ConfigResource.Type.BROKER, "")
- val op2 = new AlterConfigOp(new ConfigEntry(KafkaConfig.LogFlushIntervalMsProp, newVal.toString()),
+ val op2 = new AlterConfigOp(new ConfigEntry(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, newVal.toString()),
SET)
admin.incrementalAlterConfigs(Map(
resource -> List(op).asJavaCollection,
diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
index 9c41211751b28..a018b5d6af48a 100755
--- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
@@ -34,6 +34,7 @@ import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.ByteUtils
import org.apache.kafka.common.{TopicPartition, requests}
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
@@ -44,7 +45,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
def generateConfigs = {
val props = TestUtils.createBrokerConfig(1, zkConnectOrNull)
- props.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false")
+ props.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
List(KafkaConfig.fromProps(props))
}
diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala
index df463bf72e5a3..84d744479d5c2 100644
--- a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala
@@ -27,6 +27,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.serialization.StringSerializer
+import org.apache.kafka.server.config.ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
@@ -53,7 +54,7 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
override protected def brokerPropertyOverrides(properties: Properties): Unit = {
super.brokerPropertyOverrides(properties)
- properties.put(KafkaConfig.LogMessageDownConversionEnableProp, "false")
+ properties.put(LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "false")
}
private def initProducer(): Unit = {
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index 8485a2bcffdd8..3d3ea217f94f5 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -75,7 +75,7 @@ import org.apache.kafka.common._
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.server.ClientMetricsManager
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer}
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1}
import org.apache.kafka.server.common.{Features, MetadataVersion}
@@ -1307,7 +1307,7 @@ class KafkaApisTest extends Logging {
when(txnCoordinator.transactionTopicConfigs).thenReturn(new Properties)
true
case _ =>
- topicConfigOverride.put(KafkaConfig.NumPartitionsProp, numBrokersNeeded.toString)
+ topicConfigOverride.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numBrokersNeeded.toString)
topicConfigOverride.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, numBrokersNeeded.toString)
false
}
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 6cf94e8705d1b..173ad75d7d37a 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -28,7 +28,6 @@ import org.apache.kafka.common.record.{CompressionType, Records}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, UNKNOWN_ADDRESS_SPEC_INSTANCE}
-import org.apache.kafka.server.config.ReplicationConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
@@ -42,7 +41,7 @@ import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, Transact
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1}
-import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs}
+import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
import org.junit.jupiter.api.function.Executable
@@ -55,7 +54,7 @@ class KafkaConfigTest {
@Test
def testLogRetentionTimeHoursProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "1")
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "1")
val cfg = KafkaConfig.fromProps(props)
assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis)
@@ -64,7 +63,7 @@ class KafkaConfigTest {
@Test
def testLogRetentionTimeMinutesProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "30")
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "30")
val cfg = KafkaConfig.fromProps(props)
assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@@ -73,7 +72,7 @@ class KafkaConfigTest {
@Test
def testLogRetentionTimeMsProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1800000")
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1800000")
val cfg = KafkaConfig.fromProps(props)
assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@@ -90,8 +89,8 @@ class KafkaConfigTest {
@Test
def testLogRetentionTimeBothMinutesAndHoursProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "30")
- props.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "1")
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "30")
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "1")
val cfg = KafkaConfig.fromProps(props)
assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@@ -100,8 +99,8 @@ class KafkaConfigTest {
@Test
def testLogRetentionTimeBothMinutesAndMsProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1800000")
- props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "10")
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1800000")
+ props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "10")
val cfg = KafkaConfig.fromProps(props)
assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@@ -603,14 +602,14 @@ class KafkaConfigTest {
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.0")
// We need to set the message format version to make the configuration valid.
- props.setProperty(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.0")
+ props.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.8.2.0")
val conf2 = KafkaConfig.fromProps(props)
assertEquals(IBP_0_8_2, conf2.interBrokerProtocolVersion)
// check that 0.8.2.0 is the same as 0.8.2.1
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.1")
// We need to set the message format version to make the configuration valid
- props.setProperty(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.1")
+ props.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.8.2.1")
val conf3 = KafkaConfig.fromProps(props)
assertEquals(IBP_0_8_2, conf3.interBrokerProtocolVersion)
@@ -664,7 +663,7 @@ class KafkaConfigTest {
@Test
def testLogRollTimeMsProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.setProperty(KafkaConfig.LogRollTimeMillisProp, "1800000")
+ props.setProperty(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, "1800000")
val cfg = KafkaConfig.fromProps(props)
assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis)
@@ -673,8 +672,8 @@ class KafkaConfigTest {
@Test
def testLogRollTimeBothMsAndHoursProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
- props.setProperty(KafkaConfig.LogRollTimeMillisProp, "1800000")
- props.setProperty(KafkaConfig.LogRollTimeHoursProp, "1")
+ props.setProperty(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, "1800000")
+ props.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "1")
val cfg = KafkaConfig.fromProps(props)
assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis)
@@ -745,7 +744,7 @@ class KafkaConfigTest {
def buildConfig(interBrokerProtocol: MetadataVersion, messageFormat: MetadataVersion): KafkaConfig = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, interBrokerProtocol.version)
- props.setProperty(KafkaConfig.LogMessageFormatVersionProp, messageFormat.version)
+ props.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, messageFormat.version)
KafkaConfig.fromProps(props)
}
@@ -824,7 +823,7 @@ class KafkaConfigTest {
case KafkaConfig.MetadataMaxIdleIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.AuthorizerClassNameProp => //ignore string
- case KafkaConfig.CreateTopicPolicyClassNameProp => //ignore string
+ case ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG => //ignore string
case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
@@ -834,21 +833,21 @@ class KafkaConfigTest {
case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.FailedAuthenticationDelayMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1")
- case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogDirsProp => // ignore string
- case KafkaConfig.LogDirProp => // ignore string
- case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1)
+ case ServerLogConfigs.NUM_PARTITIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore string
+ case ServerLogConfigs.LOG_DIR_CONFIG => // ignore string
+ case ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1)
- case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
- case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0")
+ case ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
+ case ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0")
case CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "1024")
case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
@@ -857,17 +856,17 @@ class KafkaConfigTest {
case CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
case CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
case CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
- case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "3")
- case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.LogFlushSchedulerIntervalMsProp => 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.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.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
- case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0")
- case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "3")
+ case ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
+ case ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
+ case ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
+ case ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
+ case ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
+ case ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
+ case ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+ case ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0")
+ case ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
@@ -1152,12 +1151,12 @@ class KafkaConfigTest {
defaults.setProperty(KafkaConfig.BrokerIdProp, "1")
defaults.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:1122")
defaults.setProperty(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3")
- defaults.setProperty(KafkaConfig.LogDirProp, "/tmp1,/tmp2")
- defaults.setProperty(KafkaConfig.LogRollTimeHoursProp, "12")
- defaults.setProperty(KafkaConfig.LogRollTimeJitterHoursProp, "11")
- defaults.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "10")
- //For LogFlushIntervalMsProp
- defaults.setProperty(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123")
+ defaults.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, "/tmp1,/tmp2")
+ defaults.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "12")
+ defaults.setProperty(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG, "11")
+ defaults.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "10")
+ //For LOG_FLUSH_INTERVAL_MS_CONFIG
+ defaults.setProperty(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG, "123")
defaults.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_COMPRESSION_CODEC_CONFIG, CompressionType.SNAPPY.id.toString)
// For MetricRecordingLevelProp
defaults.setProperty(KafkaConfig.MetricRecordingLevelProp, Sensor.RecordingLevel.DEBUG.toString)
@@ -1493,7 +1492,7 @@ class KafkaConfigTest {
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir)
- props.setProperty(KafkaConfig.LogDirProp, dataDir)
+ props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, dataDir)
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
KafkaConfig.fromProps(props)
@@ -1511,7 +1510,7 @@ class KafkaConfigTest {
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
- props.setProperty(KafkaConfig.LogDirProp, s"$dataDir1,$dataDir2")
+ props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, s"$dataDir1,$dataDir2")
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
KafkaConfig.fromProps(props)
@@ -1861,7 +1860,7 @@ class KafkaConfigTest {
def testMultipleLogDirectoriesNotSupportedWithRemoteLogStorage(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true))
- props.put(KafkaConfig.LogDirsProp, "/tmp/a,/tmp/b")
+ props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a,/tmp/b")
val caught = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
assertTrue(caught.getMessage.contains("Multiple log directories `/tmp/a,/tmp/b` are not supported when remote log storage is enabled"))
@@ -1871,7 +1870,7 @@ class KafkaConfigTest {
def testSingleLogDirectoryWithRemoteLogStorage(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true))
- props.put(KafkaConfig.LogDirsProp, "/tmp/a")
+ props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a")
assertDoesNotThrow(() => KafkaConfig.fromProps(props))
}
}
diff --git a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala
index c0a00c42ce2c2..87aacaff4be36 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala
@@ -24,7 +24,7 @@ import org.apache.kafka.common.{KafkaException, Uuid}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions._
@@ -90,7 +90,7 @@ class KafkaRaftServerTest {
try {
writeMetaProperties(tempLogDir, metaProperties)
metadataVersion.foreach(mv => writeBootstrapMetadata(tempLogDir, mv))
- configProperties.put(KafkaConfig.LogDirProp, tempLogDir.getAbsolutePath)
+ configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, tempLogDir.getAbsolutePath)
val config = KafkaConfig.fromProps(configProperties)
KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, "")
} finally {
@@ -130,7 +130,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
- configProperties.put(KafkaConfig.LogDirProp, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
+ configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
@@ -158,7 +158,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.MetadataLogDirProp, invalidDir.getAbsolutePath)
- configProperties.put(KafkaConfig.LogDirProp, validDir.getAbsolutePath)
+ configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, validDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
@@ -188,7 +188,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, validDir.getAbsolutePath)
- configProperties.put(KafkaConfig.LogDirProp, invalidDir.getAbsolutePath)
+ configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, invalidDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
@@ -224,7 +224,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, metadataDir.getAbsolutePath)
- configProperties.put(KafkaConfig.LogDirProp, dataDir.getAbsolutePath)
+ configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, dataDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
@@ -251,7 +251,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
- configProperties.put(KafkaConfig.LogDirProp, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
+ configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
@@ -307,7 +307,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093")
configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
- configProperties.put(KafkaConfig.LogDirProp, logDir.getAbsolutePath)
+ configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
val (metaPropertiesEnsemble, bootstrapMetadata) =
invokeLoadMetaProperties(metaProperties, configProperties, None)
diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
index e507ad4a71c9f..53bd2b0e0da4b 100644
--- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
@@ -28,7 +28,7 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{KafkaStorageException, NotLeaderOrFollowerException}
import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.provider.ValueSource
@@ -86,7 +86,7 @@ class LogDirFailureTest extends IntegrationTestHarness {
try {
val props = TestUtils.createBrokerConfig(brokerCount, zkConnect, logDirCount = 3)
props.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.11.0")
- props.put(KafkaConfig.LogMessageFormatVersionProp, "0.11.0")
+ props.put(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
val kafkaConfig = KafkaConfig.fromProps(props)
val logDir = new File(kafkaConfig.logDirs.head)
// Make log directory of the partition on the leader broker inaccessible by replacing it with a file
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
index 83773511f85d6..a576c7ad5e778 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
@@ -40,7 +40,7 @@ import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.metadata.LeaderRecoveryState
import org.apache.kafka.metadata.PartitionRegistration
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesVersion}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.util.{MockTime, ShutdownableThread}
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel}
@@ -170,7 +170,7 @@ class ReplicaManagerConcurrencyTest extends Logging {
props.put(KafkaConfig.ProcessRolesProp, "broker")
props.put(KafkaConfig.NodeIdProp, localId.toString)
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
- props.put(KafkaConfig.LogDirProp, logDir.getAbsolutePath)
+ props.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
props.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, 5000.toString)
val config = new KafkaConfig(props, doLog = false)
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index ef74945990a8b..7a3047781aebd 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -76,7 +76,7 @@ import org.apache.kafka.common.config.{AbstractConfig, TopicConfig}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.raft.RaftConfig
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.timer.MockTimer
import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile
import org.mockito.invocation.InvocationOnMock
@@ -4794,7 +4794,7 @@ class ReplicaManagerTest {
val topicPartition = new TopicPartition(topic, 0)
val replicaManager = setupReplicaManagerWithMockedPurgatories(
timer = new MockTimer(time),
- propsModifier = props => props.put(KafkaConfig.LogDirsProp, dataDir.getAbsolutePath)
+ propsModifier = props => props.put(ServerLogConfigs.LOG_DIRS_CONFIG, dataDir.getAbsolutePath)
)
try {
@@ -5606,7 +5606,7 @@ class ReplicaManagerTest {
val replicaManager = setupReplicaManagerWithMockedPurgatories(
timer = new MockTimer(time),
brokerId = localId,
- propsModifier = props => props.put(KafkaConfig.LogDirsProp, dataDir.getAbsolutePath),
+ propsModifier = props => props.put(ServerLogConfigs.LOG_DIRS_CONFIG, dataDir.getAbsolutePath),
enableRemoteStorage = true
)
diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
index 551cf6314d14c..8151ece894d90 100644
--- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
@@ -38,7 +38,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.BrokerState
-import org.apache.kafka.server.config.ZkConfigs
+import org.apache.kafka.server.config.{ServerLogConfigs, ZkConfigs}
import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.function.Executable
@@ -64,11 +64,11 @@ class ServerShutdownTest extends KafkaServerTestHarness {
priorConfig.foreach { config =>
// keep the same log directory
val originals = config.originals
- val logDirsValue = originals.get(KafkaConfig.LogDirsProp)
+ val logDirsValue = originals.get(ServerLogConfigs.LOG_DIRS_CONFIG)
if (logDirsValue != null) {
- propsToChangeUponRestart.put(KafkaConfig.LogDirsProp, logDirsValue)
+ propsToChangeUponRestart.put(ServerLogConfigs.LOG_DIRS_CONFIG, logDirsValue)
} else {
- propsToChangeUponRestart.put(KafkaConfig.LogDirProp, originals.get(KafkaConfig.LogDirProp))
+ propsToChangeUponRestart.put(ServerLogConfigs.LOG_DIR_CONFIG, originals.get(ServerLogConfigs.LOG_DIR_CONFIG))
}
}
priorConfig = Some(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, zkConnectOrNull).head, propsToChangeUponRestart))
diff --git a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
index 31ba7493ba94d..8ee91c592cc86 100644
--- a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
@@ -19,7 +19,7 @@ package kafka.server.epoch
import kafka.log.UnifiedLog
import kafka.server.KafkaConfig._
-import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness}
+import kafka.server.{KafkaServer, QuorumTestHarness}
import kafka.tools.DumpLogSegments
import kafka.utils.TestUtils._
import kafka.utils.{CoreUtils, Logging, TestUtils}
@@ -28,7 +28,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.serialization.ByteArrayDeserializer
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
import org.apache.kafka.storage.internals.log.EpochEntry
@@ -178,7 +178,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
assertEquals(getLogFile(brokers(0), 0).length, getLogFile(brokers(1), 0).length, "Log files should match Broker0 vs Broker 1")
}
- //We can reproduce the pre-KIP-101 failure of this test by setting KafkaConfig.INTER_BROKER_PROTOCOL_VERSION_PROP = IBP_0_11_0_IV1
+ //We can reproduce the pre-KIP-101 failure of this test by setting ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG = IBP_0_11_0_IV1
@Test
def offsetsShouldNotGoBackwards(): Unit = {
@@ -303,7 +303,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
// A single partition topic with 2 replicas, min.isr = 1
TestUtils.createTopic(zkClient, topic, Map(0 -> Seq(100, 101)), brokers,
- CoreUtils.propsWith((KafkaConfig.MinInSyncReplicasProp, "1")))
+ CoreUtils.propsWith((ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "1")))
producer = TestUtils.createProducer(plaintextBootstrapServers(brokers), acks = 1)
diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
index 3d2ace78831ba..b1c49c6077a17 100644
--- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
@@ -37,6 +37,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.MetadataRecordSerde
import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch}
import org.apache.kafka.server.common.ApiMessageAndVersion
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.MockTime
import org.apache.kafka.snapshot.RecordsSnapshotWriter
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
@@ -315,7 +316,7 @@ class DumpLogSegmentsTest {
retentionMillis = 60 * 1000,
maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES,
maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES,
- fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS,
+ fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
nodeId = 1
)
)
diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
index a44ac0c63ad59..f9d44a8bda21d 100644
--- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
@@ -31,6 +31,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.common.metadata.UserScramCredentialRecord
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue}
import org.junit.jupiter.api.{Test, Timeout}
import org.junit.jupiter.params.ParameterizedTest
@@ -44,7 +45,7 @@ class StorageToolTest {
private def newSelfManagedProperties() = {
val properties = new Properties()
- properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo,/tmp/bar")
+ properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo,/tmp/bar")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller")
properties.setProperty(KafkaConfig.NodeIdProp, "2")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9092")
@@ -407,7 +408,7 @@ Found problem:
val propsFile = TestUtils.tempFile()
val propsStream = Files.newOutputStream(propsFile.toPath)
// This test does format the directory specified so use a tempdir
- properties.setProperty(KafkaConfig.LogDirsProp, TestUtils.tempDir().toString)
+ properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString)
properties.store(propsStream, "config.props")
propsStream.close()
@@ -461,7 +462,7 @@ Found problem:
val propsFile = TestUtils.tempFile()
val propsStream = Files.newOutputStream(propsFile.toPath)
try {
- properties.setProperty(KafkaConfig.LogDirsProp, TestUtils.tempDir().toString)
+ properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString)
properties.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, enableUnstable.toString)
properties.store(propsStream, "config.props")
} finally {
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 11e96f773a22b..d9383820fcada 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -75,7 +75,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.metadata.properties.MetaProperties
import org.apache.kafka.server.{ClientMetricsManager, ControllerRequestCompletionHandler}
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer}
-import org.apache.kafka.server.config.ReplicationConfigs
+import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
import org.apache.kafka.server.config.ZkConfigs
import org.apache.kafka.server.metrics.KafkaYammerMetrics
@@ -351,9 +351,9 @@ object TestUtils extends Logging {
// We can verify this by using a mixture of relative path and absolute path as log directories in the test
if (i % 2 == 0) tempDir().getAbsolutePath else tempRelativeDir("data")
).mkString(",")
- props.put(KafkaConfig.LogDirsProp, logDirs)
+ props.put(ServerLogConfigs.LOG_DIRS_CONFIG, logDirs)
} else {
- props.put(KafkaConfig.LogDirProp, tempDir().getAbsolutePath)
+ props.put(ServerLogConfigs.LOG_DIR_CONFIG, tempDir().getAbsolutePath)
}
if (zkConnect == null) {
props.put(KafkaConfig.ProcessRolesProp, "broker")
@@ -370,7 +370,7 @@ object TestUtils extends Logging {
props.put(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, "1500")
props.put(KafkaConfig.ControlledShutdownEnableProp, enableControlledShutdown.toString)
props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString)
- props.put(KafkaConfig.LogDeleteDelayMsProp, "1000")
+ props.put(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000")
props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100")
props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152")
props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1")
@@ -396,7 +396,7 @@ object TestUtils extends Logging {
if (enableToken)
props.put(KafkaConfig.DelegationTokenSecretKeyProp, "secretkey")
- props.put(KafkaConfig.NumPartitionsProp, numPartitions.toString)
+ props.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numPartitions.toString)
props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, defaultReplicationFactor.toString)
if (enableFetchFromFollower) {
@@ -411,7 +411,7 @@ object TestUtils extends Logging {
config.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, version.version)
// for clarity, only set the log message format version if it's not ignored
if (!LogConfig.shouldIgnoreMessageFormatVersion(version))
- config.setProperty(KafkaConfig.LogMessageFormatVersionProp, version.version)
+ config.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, version.version)
}
def createAdminClient[B <: KafkaBroker](
diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java
new file mode 100644
index 0000000000000..dcd5e8ea2dd43
--- /dev/null
+++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.server.config;
+
+import org.apache.kafka.common.config.TopicConfig;
+
+import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1;
+import static org.apache.kafka.server.config.ServerTopicConfigSynonyms.LOG_PREFIX;
+
+/**
+ * Common home for broker-side log configs which need to be accessible from the libraries shared
+ * between the broker and the multiple modules in Kafka.
+ *
+ * Note this is an internal API and subject to change without notice.
+ */
+public class ServerLogConfigs {
+ public static final String NUM_PARTITIONS_CONFIG = "num.partitions";
+ public static final int NUM_PARTITIONS_DEFAULT = 1;
+ public static final String NUM_PARTITIONS_DOC = "The default number of log partitions per topic";
+
+ public static final String LOG_DIRS_CONFIG = LOG_PREFIX + "dirs";
+ public static final String LOG_DIR_CONFIG = LOG_PREFIX + "dir";
+ public static final String LOG_DIR_DEFAULT = "/tmp/kafka-logs";
+ public static final String LOG_DIR_DOC = "The directory in which the log data is kept (supplemental for " + LOG_DIRS_CONFIG + " property)";
+ public static final String LOG_DIRS_DOC = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LOG_DIR_CONFIG + " is used.";
+
+ public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG);
+ public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file";
+
+ public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG);
+ public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours";
+ public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used";
+ public static final String LOG_ROLL_TIME_HOURS_DOC = "The maximum time before a new log segment is rolled out (in hours), secondary to " + LOG_ROLL_TIME_MILLIS_CONFIG + " property";
+
+ public static final String LOG_ROLL_TIME_JITTER_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_JITTER_MS_CONFIG);
+ public static final String LOG_ROLL_TIME_JITTER_HOURS_CONFIG = LOG_PREFIX + "roll.jitter.hours";
+ public static final String LOG_ROLL_TIME_JITTER_MILLIS_DOC = "The maximum jitter to subtract from logRollTimeMillis (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_JITTER_HOURS_CONFIG + " is used";
+ public static final String LOG_ROLL_TIME_JITTER_HOURS_DOC = "The maximum jitter to subtract from logRollTimeMillis (in hours), secondary to " + LOG_ROLL_TIME_JITTER_MILLIS_CONFIG + " property";
+
+
+ public static final String LOG_RETENTION_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_MS_CONFIG);
+ public static final String LOG_RETENTION_TIME_MINUTES_CONFIG = LOG_PREFIX + "retention.minutes";
+ public static final String LOG_RETENTION_TIME_HOURS_CONFIG = LOG_PREFIX + "retention.hours";
+ public static final String LOG_RETENTION_TIME_MILLIS_DOC = "The number of milliseconds to keep a log file before deleting it (in milliseconds), If not set, the value in " + LOG_RETENTION_TIME_MINUTES_CONFIG + " is used. If set to -1, no time limit is applied.";
+ public static final String LOG_RETENTION_TIME_MINUTES_DOC = "The number of minutes to keep a log file before deleting it (in minutes), secondary to " + LOG_RETENTION_TIME_MILLIS_CONFIG + " property. If not set, the value in " + LOG_RETENTION_TIME_HOURS_CONFIG + " is used";
+ public static final String LOG_RETENTION_TIME_HOURS_DOC = "The number of hours to keep a log file before deleting it (in hours), tertiary to " + LOG_RETENTION_TIME_MILLIS_CONFIG + " property";
+
+ public static final String LOG_RETENTION_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_BYTES_CONFIG);
+ public static final long LOG_RETENTION_BYTES_DEFAULT = -1L;
+ public static final String LOG_RETENTION_BYTES_DOC = "The maximum size of the log before deleting it";
+
+ public static final String LOG_CLEANUP_INTERVAL_MS_CONFIG = LOG_PREFIX + "retention.check.interval.ms";
+ public static final long LOG_CLEANUP_INTERVAL_MS_DEFAULT = 5 * 60 * 1000L;
+ public static final String LOG_CLEANUP_INTERVAL_MS_DOC = "The frequency in milliseconds that the log cleaner checks whether any log is eligible for deletion";
+
+ public static final String LOG_CLEANUP_POLICY_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG);
+ public static final String LOG_CLEANUP_POLICY_DEFAULT = TopicConfig.CLEANUP_POLICY_DELETE;
+ public static final String LOG_CLEANUP_POLICY_DOC = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies. Valid policies are: \"delete\" and \"compact\"";
+
+ public static final String LOG_INDEX_SIZE_MAX_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG);
+ public static final int LOG_INDEX_SIZE_MAX_BYTES_DEFAULT = 10 * 1024 * 1024;
+ public static final String LOG_INDEX_SIZE_MAX_BYTES_DOC = "The maximum size in bytes of the offset index";
+
+ public static final String LOG_INDEX_INTERVAL_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG);
+ public static final int LOG_INDEX_INTERVAL_BYTES_DEFAULT = 4096;
+ public static final String LOG_INDEX_INTERVAL_BYTES_DOC = "The interval with which we add an entry to the offset index.";
+
+ public static final String LOG_FLUSH_INTERVAL_MESSAGES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG);
+ public static final long LOG_FLUSH_INTERVAL_MESSAGES_DEFAULT = Long.MAX_VALUE;
+ public static final String LOG_FLUSH_INTERVAL_MESSAGES_DOC = "The number of messages accumulated on a log partition before messages are flushed to disk.";
+
+ public static final String LOG_DELETE_DELAY_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG);
+ public static final long LOG_DELETE_DELAY_MS_DEFAULT = 60000L;
+ public static final String LOG_DELETE_DELAY_MS_DOC = "The amount of time to wait before deleting a file from the filesystem";
+
+ public static final String LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG = LOG_PREFIX + "flush.scheduler.interval.ms";
+ public static final long LOG_FLUSH_SCHEDULER_INTERVAL_MS_DEFAULT = Long.MAX_VALUE;
+ public static final String LOG_FLUSH_SCHEDULER_INTERVAL_MS_DOC = "The frequency in ms that the log flusher checks whether any log needs to be flushed to disk";
+
+ public static final String LOG_FLUSH_INTERVAL_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MS_CONFIG);
+ public static final String LOG_FLUSH_INTERVAL_MS_DOC = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk. If not set, the value in " + LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG + " is used";
+
+ public static final String LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG = LOG_PREFIX + "flush.offset.checkpoint.interval.ms";
+ public static final int LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT = 60000;
+ public static final String LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DOC = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point.";
+
+ public static final String LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG = LOG_PREFIX + "flush.start.offset.checkpoint.interval.ms";
+ public static final int LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT = 60000;
+ public static final String LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DOC = "The frequency with which we update the persistent record of log start offset";
+
+ public static final String LOG_PRE_ALLOCATE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.PREALLOCATE_CONFIG);
+ public static final String LOG_PRE_ALLOCATE_ENABLE_DOC = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true.";
+
+ /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
+ /**
+ * @deprecated since "3.0"
+ */
+ @Deprecated
+ public static final String LOG_MESSAGE_FORMAT_VERSION_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG);
+ @Deprecated
+ public static final String LOG_MESSAGE_FORMAT_VERSION_DEFAULT = IBP_3_0_IV1.version();
+ public static final String LOG_MESSAGE_FORMAT_VERSION_DOC = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid MetadataVersion. " +
+ "Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check MetadataVersion for more details. By setting a particular message format version, the " +
+ "user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " +
+ "will cause consumers with older versions to break as they will receive messages with a format that they don't understand.";
+
+ public static final String LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG);
+ public static final String LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT = "CreateTime";
+ public static final String LOG_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
.";
+
+ /* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
+ /**
+ * @deprecated since "3.6"
+ */
+ @Deprecated
+ public static final String LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG);
+ @Deprecated
+ public static final long LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT = Long.MAX_VALUE;
+ public static final String LOG_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 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." +
+ "The maximum timestamp difference allowed should be no greater than log.retention.ms to avoid unnecessarily frequent log rolling.";
+
+ public static final String LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG);
+ public static final long LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT = Long.MAX_VALUE;
+ public static final String LOG_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 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.";
+ public static final String LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG);
+ public static final long LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT = Long.MAX_VALUE;
+ public static final String LOG_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 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.";
+
+ public static final String NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG = "num.recovery.threads.per.data.dir";
+ public static final int NUM_RECOVERY_THREADS_PER_DATA_DIR_DEFAULT = 1;
+ public static final String NUM_RECOVERY_THREADS_PER_DATA_DIR_DOC = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown";
+
+ public static final String AUTO_CREATE_TOPICS_ENABLE_CONFIG = "auto.create.topics.enable";
+ public static final boolean AUTO_CREATE_TOPICS_ENABLE_DEFAULT = true;
+ public static final String AUTO_CREATE_TOPICS_ENABLE_DOC = "Enable auto creation of topic on the server.";
+
+ public static final String MIN_IN_SYNC_REPLICAS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG);
+ public static final int MIN_IN_SYNC_REPLICAS_DEFAULT = 1;
+ public static final String MIN_IN_SYNC_REPLICAS_DOC = "When a producer sets acks to \"all\" (or \"-1\"), " +
+ "min.insync.replicas
specifies the minimum number of replicas that must acknowledge " +
+ "a write for the write to be considered successful. If this minimum cannot be met, " +
+ "then the producer will raise an exception (either NotEnoughReplicas
or " +
+ "NotEnoughReplicasAfterAppend
).
When used together, min.insync.replicas
and acks " +
+ "allow you to enforce greater durability guarantees. A typical scenario would be to " +
+ "create a topic with a replication factor of 3, set min.insync.replicas
to 2, and " +
+ "produce with acks of \"all\". This will ensure that the producer raises an exception " +
+ "if a majority of replicas do not receive a write.";
+
+ public static final String CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG = "create.topic.policy.class.name";
+ public static final String CREATE_TOPIC_POLICY_CLASS_NAME_DOC = "The create topic policy class that should be used for validation. The class should " +
+ "implement the org.apache.kafka.server.policy.CreateTopicPolicy
interface.";
+ public static final String ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG = "alter.config.policy.class.name";
+ public static final String ALTER_CONFIG_POLICY_CLASS_NAME_DOC = "The alter configs policy class that should be used for validation. The class should " +
+ "implement the org.apache.kafka.server.policy.AlterConfigPolicy
interface.";
+ public static final String LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG);
+ public static final boolean LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT = true;
+ public static final String LOG_MESSAGE_DOWNCONVERSION_ENABLE_DOC = "This configuration controls whether " +
+ "down-conversion of message formats is enabled to satisfy consume requests. When set to false
, " +
+ "broker will not perform down-conversion for consumers expecting an older message format. The broker responds " +
+ "with UNSUPPORTED_VERSION
error for consume requests from such older clients. This configuration" +
+ "does not apply to any message format conversion that might be required for replication to followers.";
+}
diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java
index 02cdc1cc875f8..9cc0f967fdb5f 100644
--- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java
+++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java
@@ -29,7 +29,7 @@
import org.apache.kafka.common.utils.Utils;
public final class ServerTopicConfigSynonyms {
- private static final String LOG_PREFIX = "log.";
+ public static final String LOG_PREFIX = "log.";
public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner.";
/**
diff --git a/server/src/main/java/org/apache/kafka/server/config/Defaults.java b/server/src/main/java/org/apache/kafka/server/config/Defaults.java
index d758d54281bbf..9337316f5eb08 100644
--- a/server/src/main/java/org/apache/kafka/server/config/Defaults.java
+++ b/server/src/main/java/org/apache/kafka/server/config/Defaults.java
@@ -77,15 +77,6 @@ public class Defaults {
public static final long CONNECTION_SETUP_TIMEOUT_MAX_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS;
public static final int FAILED_AUTHENTICATION_DELAY_MS = 100;
- /** ********* Log Configuration *********/
- public static final int NUM_PARTITIONS = 1;
- public static final String LOG_DIR = "/tmp/kafka-logs";
- public static final long LOG_CLEANUP_INTERVAL_MS = 5 * 60 * 1000L;
- public static final int LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS = 60000;
- public static final int LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS = 60000;
- public static final int NUM_RECOVERY_THREADS_PER_DATA_DIR = 1;
- public static final boolean AUTO_CREATE_TOPICS_ENABLE = true;
-
/** ********* Controlled shutdown configuration *********/
public static final int CONTROLLED_SHUTDOWN_MAX_RETRIES = 3;
public static final int CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS = 5000;
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java
index 13aa13a7031d7..174987e6bccb4 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java
@@ -56,6 +56,7 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.MetadataVersionValidator;
+import org.apache.kafka.server.config.ServerLogConfigs;
import org.apache.kafka.server.config.ServerTopicConfigSynonyms;
import org.apache.kafka.server.record.BrokerCompressionType;
@@ -159,29 +160,15 @@ public Optional serverConfigName(String configName) {
public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L;
public static final long DEFAULT_SEGMENT_JITTER_MS = 0;
public static final long DEFAULT_RETENTION_MS = 24 * 7 * 60 * 60 * 1000L;
- public static final long DEFAULT_RETENTION_BYTES = -1L;
- public static final int DEFAULT_SEGMENT_INDEX_BYTES = 10 * 1024 * 1024;
- public static final int DEFAULT_INDEX_INTERVAL_BYTES = 4096;
- public static final long DEFAULT_FILE_DELETE_DELAY_MS = 60000L;
- public static final String DEFAULT_CLEANUP_POLICY = TopicConfig.CLEANUP_POLICY_DELETE;
- public static final long DEFAULT_FLUSH_MESSAGES_INTERVAL = Long.MAX_VALUE;
- public static final long DEFAULT_FLUSH_MS = Long.MAX_VALUE;
public static final long DEFAULT_DELETE_RETENTION_MS = 24 * 60 * 60 * 1000L;
public static final long DEFAULT_MIN_COMPACTION_LAG_MS = 0;
public static final long DEFAULT_MAX_COMPACTION_LAG_MS = Long.MAX_VALUE;
public static final double DEFAULT_MIN_CLEANABLE_DIRTY_RATIO = 0.5;
public static final boolean DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE = false;
- public static final int DEFAULT_MIN_IN_SYNC_REPLICAS = 1;
public static final String DEFAULT_COMPRESSION_TYPE = BrokerCompressionType.PRODUCER.name;
public static final boolean DEFAULT_PREALLOCATE = false;
- 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_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 long DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS = ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT;
public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false;
public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes
@@ -189,9 +176,10 @@ public Optional serverConfigName(String configName) {
public static final List DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList();
public static final List DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList();
- /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
+ /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details
+ * Keep DEFAULT_MESSAGE_FORMAT_VERSION as a way to handlee the deprecated value */
@Deprecated
- public static final String DEFAULT_MESSAGE_FORMAT_VERSION = IBP_3_0_IV1.version();
+ public static final String DEFAULT_MESSAGE_FORMAT_VERSION = ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT;
// Leave these out of TopicConfig for now as they are replication quota configs
public static final String LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG = "leader.replication.throttled.replicas";
@@ -234,20 +222,20 @@ public Optional serverConfigName(String configName) {
.define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC)
.define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM,
TopicConfig.SEGMENT_JITTER_MS_DOC)
- .define(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, INT, DEFAULT_SEGMENT_INDEX_BYTES, atLeast(4), MEDIUM,
+ .define(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT, atLeast(4), MEDIUM,
TopicConfig.SEGMENT_INDEX_BYTES_DOC)
- .define(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, LONG, DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), MEDIUM,
+ .define(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_DEFAULT, atLeast(1), MEDIUM,
TopicConfig.FLUSH_MESSAGES_INTERVAL_DOC)
- .define(TopicConfig.FLUSH_MS_CONFIG, LONG, DEFAULT_FLUSH_MS, atLeast(0), MEDIUM,
+ .define(TopicConfig.FLUSH_MS_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.FLUSH_MS_DOC)
// can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize
- .define(TopicConfig.RETENTION_BYTES_CONFIG, LONG, DEFAULT_RETENTION_BYTES, MEDIUM, TopicConfig.RETENTION_BYTES_DOC)
+ .define(TopicConfig.RETENTION_BYTES_CONFIG, LONG, ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, MEDIUM, TopicConfig.RETENTION_BYTES_DOC)
// can be negative. See kafka.log.LogManager.cleanupExpiredSegments
.define(TopicConfig.RETENTION_MS_CONFIG, LONG, DEFAULT_RETENTION_MS, atLeast(-1), MEDIUM,
TopicConfig.RETENTION_MS_DOC)
.define(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, INT, DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), MEDIUM,
TopicConfig.MAX_MESSAGE_BYTES_DOC)
- .define(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, INT, DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM,
+ .define(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.INDEX_INTERVAL_BYTES_DOC)
.define(TopicConfig.DELETE_RETENTION_MS_CONFIG, LONG, DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM,
TopicConfig.DELETE_RETENTION_MS_DOC)
@@ -255,34 +243,34 @@ public Optional serverConfigName(String configName) {
TopicConfig.MIN_COMPACTION_LAG_MS_DOC)
.define(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, LONG, DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM,
TopicConfig.MAX_COMPACTION_LAG_MS_DOC)
- .define(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, LONG, DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), MEDIUM,
+ .define(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.FILE_DELETE_DELAY_MS_DOC)
.define(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, DOUBLE, DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM,
TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_DOC)
- .define(TopicConfig.CLEANUP_POLICY_CONFIG, LIST, DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT,
+ .define(TopicConfig.CLEANUP_POLICY_CONFIG, LIST, ServerLogConfigs.LOG_CLEANUP_POLICY_DEFAULT, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT,
TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, TopicConfig.CLEANUP_POLICY_DOC)
.define(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, BOOLEAN, DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE,
MEDIUM, TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_DOC)
- .define(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, INT, DEFAULT_MIN_IN_SYNC_REPLICAS, atLeast(1), MEDIUM,
+ .define(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, INT, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT, atLeast(1), MEDIUM,
TopicConfig.MIN_IN_SYNC_REPLICAS_DOC)
.define(TopicConfig.COMPRESSION_TYPE_CONFIG, STRING, DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names().toArray(new String[0])),
MEDIUM, TopicConfig.COMPRESSION_TYPE_DOC)
.define(TopicConfig.PREALLOCATE_CONFIG, BOOLEAN, DEFAULT_PREALLOCATE, MEDIUM, TopicConfig.PREALLOCATE_DOC)
.define(MESSAGE_FORMAT_VERSION_CONFIG, STRING, DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM,
MESSAGE_FORMAT_VERSION_DOC)
- .define(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, DEFAULT_MESSAGE_TIMESTAMP_TYPE,
+ .define(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT,
in("CreateTime", "LogAppendTime"), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC)
.define(MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS,
atLeast(0), MEDIUM, MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
- .define(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_BEFORE_MAX_MS,
+ .define(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT,
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,
+ .define(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT,
atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC)
.define(LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS,
ThrottledReplicaListValidator.INSTANCE, MEDIUM, LEADER_REPLICATION_THROTTLED_REPLICAS_DOC)
.define(FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS,
ThrottledReplicaListValidator.INSTANCE, MEDIUM, FOLLOWER_REPLICATION_THROTTLED_REPLICAS_DOC)
- .define(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW,
+ .define(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT, LOW,
TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC)
.define(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_STORAGE_ENABLE, null,
MEDIUM, TopicConfig.REMOTE_LOG_STORAGE_ENABLE_DOC)
diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java
index f3fb081dd4d98..2e949887956fc 100644
--- a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java
+++ b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/TieredStorageTestUtils.java
@@ -16,7 +16,6 @@
*/
package org.apache.kafka.tiered.storage.utils;
-import kafka.server.KafkaConfig;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.TopicConfig;
@@ -49,6 +48,7 @@
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CONFIG_PREFIX_PROP;
+import static org.apache.kafka.server.config.ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG;
public class TieredStorageTestUtils {
@@ -137,7 +137,7 @@ public static Properties createPropsForRemoteStorage(String testClassName,
// the integration tests can confirm a given log segment is present only in the second-tier storage.
// Note that this does not impact the eligibility of a log segment to be offloaded to the
// second-tier storage.
- overridingProps.setProperty(KafkaConfig.LogCleanupIntervalMsProp(), LOG_CLEANUP_INTERVAL_MS.toString());
+ overridingProps.setProperty(LOG_CLEANUP_INTERVAL_MS_CONFIG, LOG_CLEANUP_INTERVAL_MS.toString());
// The directory of the second-tier storage needs to be constant across all instances of storage managers
// in every broker and throughout the test. Indeed, as brokers are restarted during the test.
// You can override this property with a fixed path of your choice if you wish to use a non-temporary
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
index 5e1b0ff09280a..c9df44caad0f9 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
@@ -25,6 +25,7 @@
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
import org.apache.kafka.server.config.ConfigType;
+import org.apache.kafka.server.config.ServerLogConfigs;
import org.apache.kafka.server.config.ZkConfigs;
import org.apache.kafka.server.util.MockTime;
import org.apache.kafka.storage.internals.log.CleanerConfig;
@@ -121,7 +122,7 @@ public void start() throws IOException {
putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) 1);
putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 5);
putIfAbsent(brokerConfig, TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 5);
- putIfAbsent(brokerConfig, KafkaConfig.AutoCreateTopicsEnableProp(), true);
+ putIfAbsent(brokerConfig, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, true);
for (int i = 0; i < brokers.length; i++) {
brokerConfig.put(KafkaConfig.BrokerIdProp(), i);
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
index b0a80a8da1f8d..d4e8b8ffbceba 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
@@ -40,6 +40,11 @@
import java.util.Properties;
import java.util.concurrent.ExecutionException;
+import static org.apache.kafka.server.config.ServerLogConfigs.NUM_PARTITIONS_CONFIG;
+import static org.apache.kafka.server.config.ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG;
+import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
+
+
/**
* Runs an in-memory, "embedded" instance of a Kafka broker, which listens at `127.0.0.1:9092` by
* default.
@@ -87,14 +92,14 @@ public KafkaEmbedded(final Properties config, final MockTime time) throws IOExce
private Properties effectiveConfigFrom(final Properties initialConfig) {
final Properties effectiveConfig = new Properties();
effectiveConfig.put(KafkaConfig.BrokerIdProp(), 0);
- effectiveConfig.put(KafkaConfig.NumPartitionsProp(), 1);
- effectiveConfig.put(KafkaConfig.AutoCreateTopicsEnableProp(), true);
+ effectiveConfig.put(NUM_PARTITIONS_CONFIG, 1);
+ effectiveConfig.put(AUTO_CREATE_TOPICS_ENABLE_CONFIG, true);
effectiveConfig.put(KafkaConfig.MessageMaxBytesProp(), 1000000);
effectiveConfig.put(KafkaConfig.ControlledShutdownEnableProp(), true);
effectiveConfig.put(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, 10000);
effectiveConfig.putAll(initialConfig);
- effectiveConfig.setProperty(KafkaConfig.LogDirProp(), logDir.getAbsolutePath());
+ effectiveConfig.setProperty(LOG_DIR_CONFIG, logDir.getAbsolutePath());
return effectiveConfig;
}