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; }