diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 32f13d0e34004..1ab694af3c964 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -234,6 +234,7 @@ + @@ -279,6 +280,7 @@ + @@ -362,7 +364,9 @@ + + diff --git a/core/src/main/java/kafka/server/builders/LogManagerBuilder.java b/core/src/main/java/kafka/server/builders/LogManagerBuilder.java index 2e5e293b120f7..2405f259f7a65 100644 --- a/core/src/main/java/kafka/server/builders/LogManagerBuilder.java +++ b/core/src/main/java/kafka/server/builders/LogManagerBuilder.java @@ -17,8 +17,6 @@ package kafka.server.builders; -import kafka.log.CleanerConfig; -import kafka.log.LogConfig; import kafka.log.LogManager; import kafka.log.ProducerStateManagerConfig; import kafka.server.BrokerTopicStats; @@ -26,6 +24,8 @@ import kafka.utils.Scheduler; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.CleanerConfig; +import org.apache.kafka.server.log.internals.LogConfig; import org.apache.kafka.server.log.internals.LogDirFailureChannel; import scala.collection.JavaConverters; diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index db149935a78b6..4821bbcd205c4 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -20,9 +20,7 @@ package kafka.admin import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} - import joptsimple._ -import kafka.log.LogConfig import kafka.server.DynamicConfig.QuotaConfigs import kafka.server.{ConfigEntityName, ConfigType, Defaults, DynamicBrokerConfig, DynamicConfig, KafkaConfig} import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit, Logging, PasswordEncoder} @@ -30,7 +28,7 @@ import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasOptions, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, DescribeConfigsOptions, ListTopicsOptions, ScramCredentialInfo, UserScramCredentialDeletion, UserScramCredentialUpsertion, Config => JConfig, ScramMechanism => PublicScramMechanism} import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.common.internals.Topic @@ -38,6 +36,7 @@ import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism} import org.apache.kafka.common.utils.{Sanitizer, Time, Utils} +import org.apache.kafka.server.log.internals.LogConfig import org.apache.zookeeper.client.ZKClientConfig import scala.annotation.nowarn @@ -293,8 +292,8 @@ object ConfigCommand extends Logging { //Create properties, parsing square brackets from values if necessary configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).replaceAll("\\[?\\]?", "").trim)) } - if (props.containsKey(LogConfig.MessageFormatVersionProp)) { - println(s"WARNING: The configuration ${LogConfig.MessageFormatVersionProp}=${props.getProperty(LogConfig.MessageFormatVersionProp)} is specified. " + + if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) { + println(s"WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " + "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " + "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.") } @@ -786,7 +785,7 @@ object ConfigCommand extends Logging { val nl = System.getProperty("line.separator") val addConfig = parser.accepts("add-config", "Key Value pairs of configs to add. Square brackets can be used to group values which contain commas: 'k1=v1,k2=[v1,v2,v2],k3=v3'. The following is a list of valid configurations: " + - "For entity-type '" + ConfigType.Topic + "': " + LogConfig.configNames.map("\t" + _).mkString(nl, nl, nl) + + "For entity-type '" + ConfigType.Topic + "': " + LogConfig.configNames.asScala.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + ConfigType.Broker + "': " + DynamicConfig.Broker.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + ConfigType.User + "': " + DynamicConfig.User.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + ConfigType.Client + "': " + DynamicConfig.Client.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 8bb92da8921ef..2f3e3a69117f3 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -19,9 +19,7 @@ package kafka.admin import java.util import java.util.Optional import java.util.concurrent.ExecutionException - import kafka.common.AdminCommandFailedException -import kafka.log.LogConfig import kafka.server.DynamicConfig import kafka.utils.{CommandDefaultOptions, CommandLineUtils, CoreUtils, Exit, Json, Logging} import kafka.utils.Implicits._ @@ -32,6 +30,7 @@ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{ReplicaNotAvailableException, UnknownTopicOrPartitionException} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, KafkaFuture, TopicPartition, TopicPartitionReplica} +import org.apache.kafka.server.log.internals.LogConfig import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq, mutable} @@ -70,9 +69,9 @@ object ReassignPartitionsCommand extends Logging { // Throttles that are set at the level of an individual topic. private[admin] val topicLevelLeaderThrottle = - LogConfig.LeaderReplicationThrottledReplicasProp + LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG private[admin] val topicLevelFollowerThrottle = - LogConfig.FollowerReplicationThrottledReplicasProp + LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG private[admin] val topicLevelThrottles = Seq( topicLevelLeaderThrottle, topicLevelFollowerThrottle diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index abecc383fbd05..65148b203f826 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -21,7 +21,6 @@ import java.util import java.util.{Collections, Properties} import joptsimple._ import kafka.common.AdminCommandFailedException -import kafka.log.LogConfig import kafka.utils._ import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.admin.CreatePartitionsOptions @@ -34,6 +33,7 @@ import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.errors.{ClusterAuthorizationException, TopicExistsException, UnsupportedVersionException} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.log.internals.LogConfig import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -435,8 +435,8 @@ object TopicCommand extends Logging { val props = new Properties configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim)) LogConfig.validate(props) - if (props.containsKey(LogConfig.MessageFormatVersionProp)) { - println(s"WARNING: The configuration ${LogConfig.MessageFormatVersionProp}=${props.getProperty(LogConfig.MessageFormatVersionProp)} is specified. " + + if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) { + println(s"WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " + "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " + "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.") } @@ -518,7 +518,7 @@ object TopicCommand extends Logging { private val kafkaConfigsCanAlterTopicConfigsViaBootstrapServer = " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)" private val configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." + - " The following is a list of valid configurations: " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + + " The following is a list of valid configurations: " + nl + LogConfig.configNames.asScala.map("\t" + _).mkString(nl) + nl + "See the Kafka documentation for full details on the topic configs." + " It is supported only in combination with --create if --bootstrap-server option is used" + kafkaConfigsCanAlterTopicConfigsViaBootstrapServer + ".") diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 4236c8bfc7327..9cc46c3da1514 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -19,10 +19,10 @@ package kafka.coordinator.group import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean import kafka.common.OffsetAndMetadata -import kafka.log.LogConfig import kafka.server._ import kafka.utils.Logging import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity @@ -88,9 +88,9 @@ class GroupCoordinator(val brokerId: Int, def offsetsTopicConfigs: Properties = { val props = new Properties - props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - props.put(LogConfig.SegmentBytesProp, offsetConfig.offsetsTopicSegmentBytes.toString) - props.put(LogConfig.CompressionTypeProp, BrokerCompressionType.PRODUCER.name) + props.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, offsetConfig.offsetsTopicSegmentBytes.toString) + props.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name) props } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 9073d491abc28..e40c2e277b599 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -16,8 +16,6 @@ */ package kafka.coordinator.transaction -import kafka.log.LogConfig - import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent.TimeUnit @@ -27,6 +25,7 @@ import kafka.server.{Defaults, FetchLogEnd, ReplicaManager, RequestLocal} import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.{Logging, Pool, Scheduler} import kafka.utils.Implicits._ +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.ListTransactionsResponseData import org.apache.kafka.common.metrics.Metrics @@ -402,11 +401,11 @@ class TransactionStateManager(brokerId: Int, val props = new Properties // enforce disabled unclean leader election, no compression types, and compact cleanup policy - props.put(LogConfig.UncleanLeaderElectionEnableProp, "false") - props.put(LogConfig.CompressionTypeProp, BrokerCompressionType.UNCOMPRESSED.name) - props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - props.put(LogConfig.MinInSyncReplicasProp, config.transactionLogMinInsyncReplicas.toString) - props.put(LogConfig.SegmentBytesProp, config.transactionLogSegmentBytes.toString) + props.put(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false") + props.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.UNCOMPRESSED.name) + props.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + props.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, config.transactionLogMinInsyncReplicas.toString) + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, config.transactionLogSegmentBytes.toString) props } diff --git a/core/src/main/scala/kafka/log/CleanerConfig.scala b/core/src/main/scala/kafka/log/CleanerConfig.scala deleted file mode 100644 index 782bc9adb2274..0000000000000 --- a/core/src/main/scala/kafka/log/CleanerConfig.scala +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.log - -/** - * Configuration parameters for the log cleaner - * - * @param numThreads The number of cleaner threads to run - * @param dedupeBufferSize The total memory used for log deduplication - * @param dedupeBufferLoadFactor The maximum percent full for the deduplication buffer - * @param maxMessageSize The maximum size of a message that can appear in the log - * @param maxIoBytesPerSecond The maximum read and write I/O that all cleaner threads are allowed to do - * @param backOffMs The amount of time to wait before rechecking if no logs are eligible for cleaning - * @param enableCleaner Allows completely disabling the log cleaner - * @param hashAlgorithm The hash algorithm to use in key comparison. - */ -case class CleanerConfig(numThreads: Int = 1, - dedupeBufferSize: Long = 4*1024*1024L, - dedupeBufferLoadFactor: Double = 0.9d, - ioBufferSize: Int = 1024*1024, - maxMessageSize: Int = 32*1024*1024, - maxIoBytesPerSecond: Double = Double.MaxValue, - backOffMs: Long = 15 * 1000, - enableCleaner: Boolean = true, - hashAlgorithm: String = "MD5") { -} diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala index 28c809309d3d0..a383bbe6024e3 100644 --- a/core/src/main/scala/kafka/log/LocalLog.scala +++ b/core/src/main/scala/kafka/log/LocalLog.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.server.log.internals.LogFileUtils.offsetFromFileName -import org.apache.kafka.server.log.internals.{AbortedTxn, LogDirFailureChannel, LogOffsetMetadata, OffsetPosition} +import org.apache.kafka.server.log.internals.{AbortedTxn, LogConfig, LogDirFailureChannel, LogOffsetMetadata, OffsetPosition} import scala.jdk.CollectionConverters._ import scala.collection.{Seq, immutable} diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 423ff8866674a..83b1b0e81b6b8 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time} -import org.apache.kafka.server.log.internals.{AbortedTxn, LogDirFailureChannel, OffsetMap, SkimpyOffsetMap, TransactionIndex} +import org.apache.kafka.server.log.internals.{AbortedTxn, CleanerConfig, LogDirFailureChannel, OffsetMap, SkimpyOffsetMap, TransactionIndex} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -174,8 +174,7 @@ class LogCleaner(initialConfig: CleanerConfig, } override def validateReconfiguration(newConfig: KafkaConfig): Unit = { - val newCleanerConfig = LogCleaner.cleanerConfig(newConfig) - val numThreads = newCleanerConfig.numThreads + val numThreads = LogCleaner.cleanerConfig(newConfig).numThreads val currentThreads = config.numThreads if (numThreads < 1) throw new ConfigException(s"Log cleaner threads should be at least 1") @@ -332,7 +331,7 @@ class LogCleaner(initialConfig: CleanerConfig, override def doWork(): Unit = { val cleaned = tryCleanFilthiestLog() if (!cleaned) - pause(config.backOffMs, TimeUnit.MILLISECONDS) + pause(config.backoffMs, TimeUnit.MILLISECONDS) cleanerManager.maintainUncleanablePartitions() } @@ -454,14 +453,14 @@ object LogCleaner { ) def cleanerConfig(config: KafkaConfig): CleanerConfig = { - CleanerConfig(numThreads = config.logCleanerThreads, - dedupeBufferSize = config.logCleanerDedupeBufferSize, - dedupeBufferLoadFactor = config.logCleanerDedupeBufferLoadFactor, - ioBufferSize = config.logCleanerIoBufferSize, - maxMessageSize = config.messageMaxBytes, - maxIoBytesPerSecond = config.logCleanerIoMaxBytesPerSecond, - backOffMs = config.logCleanerBackoffMs, - enableCleaner = config.logCleanerEnable) + new CleanerConfig(config.logCleanerThreads, + config.logCleanerDedupeBufferSize, + config.logCleanerDedupeBufferLoadFactor, + config.logCleanerIoBufferSize, + config.messageMaxBytes, + config.logCleanerIoMaxBytesPerSecond, + config.logCleanerBackoffMs, + config.logCleanerEnable) } } diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala deleted file mode 100755 index a4d489bda45b1..0000000000000 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ /dev/null @@ -1,591 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.log - -import kafka.log.LogConfig.configDef -import kafka.server.{KafkaConfig, ThrottledReplicaListValidator} -import kafka.utils.Implicits._ -import org.apache.kafka.common.config.ConfigDef.{ConfigKey, ValidList, Validator} -import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, TopicConfig} -import org.apache.kafka.common.errors.InvalidConfigurationException -import org.apache.kafka.common.record.{LegacyRecord, RecordVersion, TimestampType} -import org.apache.kafka.common.utils.{ConfigUtils, Utils} -import org.apache.kafka.metadata.ConfigSynonym -import org.apache.kafka.metadata.ConfigSynonym.{HOURS_TO_MILLISECONDS, MINUTES_TO_MILLISECONDS} - -import java.util.Arrays.asList -import java.util.{Collections, Locale, Properties} -import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator} -import org.apache.kafka.server.common.MetadataVersion._ -import org.apache.kafka.server.record.BrokerCompressionType - -import scala.annotation.nowarn -import scala.collection.{Map, mutable} -import scala.jdk.CollectionConverters._ - -object Defaults { - val SegmentSize = kafka.server.Defaults.LogSegmentBytes - val SegmentMs = kafka.server.Defaults.LogRollHours * 60 * 60 * 1000L - val SegmentJitterMs = kafka.server.Defaults.LogRollJitterHours * 60 * 60 * 1000L - val FlushInterval = kafka.server.Defaults.LogFlushIntervalMessages - val FlushMs = kafka.server.Defaults.LogFlushSchedulerIntervalMs - val RetentionSize = kafka.server.Defaults.LogRetentionBytes - val RetentionMs = kafka.server.Defaults.LogRetentionHours * 60 * 60 * 1000L - val RemoteLogStorageEnable = false - val LocalRetentionBytes = -2 // It indicates the value to be derived from RetentionSize - val LocalRetentionMs = -2 // It indicates the value to be derived from RetentionMs - val MaxMessageSize = kafka.server.Defaults.MessageMaxBytes - val MaxIndexSize = kafka.server.Defaults.LogIndexSizeMaxBytes - val IndexInterval = kafka.server.Defaults.LogIndexIntervalBytes - val FileDeleteDelayMs = kafka.server.Defaults.LogDeleteDelayMs - val DeleteRetentionMs = kafka.server.Defaults.LogCleanerDeleteRetentionMs - val MinCompactionLagMs = kafka.server.Defaults.LogCleanerMinCompactionLagMs - val MaxCompactionLagMs = kafka.server.Defaults.LogCleanerMaxCompactionLagMs - val MinCleanableDirtyRatio = kafka.server.Defaults.LogCleanerMinCleanRatio - val CleanupPolicy = kafka.server.Defaults.LogCleanupPolicy - val UncleanLeaderElectionEnable = kafka.server.Defaults.UncleanLeaderElectionEnable - val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas - val CompressionType = kafka.server.Defaults.CompressionType - val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable - - /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ - @deprecated("3.0") - val MessageFormatVersion = kafka.server.Defaults.LogMessageFormatVersion - - val MessageTimestampType = kafka.server.Defaults.LogMessageTimestampType - val MessageTimestampDifferenceMaxMs = kafka.server.Defaults.LogMessageTimestampDifferenceMaxMs - val LeaderReplicationThrottledReplicas = Collections.emptyList[String]() - val FollowerReplicationThrottledReplicas = Collections.emptyList[String]() - val MessageDownConversionEnable = kafka.server.Defaults.MessageDownConversionEnable -} - -case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String] = Set.empty) - extends AbstractConfig(LogConfig.configDef, props, false) { - /** - * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig - * should also go in [[LogConfig.extractLogConfigMap()]]. - */ - val segmentSize = getInt(LogConfig.SegmentBytesProp) - val segmentMs = getLong(LogConfig.SegmentMsProp) - val segmentJitterMs = getLong(LogConfig.SegmentJitterMsProp) - val maxIndexSize = getInt(LogConfig.SegmentIndexBytesProp) - val flushInterval = getLong(LogConfig.FlushMessagesProp) - val flushMs = getLong(LogConfig.FlushMsProp) - val retentionSize = getLong(LogConfig.RetentionBytesProp) - val retentionMs = getLong(LogConfig.RetentionMsProp) - val maxMessageSize = getInt(LogConfig.MaxMessageBytesProp) - val indexInterval = getInt(LogConfig.IndexIntervalBytesProp) - val fileDeleteDelayMs = getLong(LogConfig.FileDeleteDelayMsProp) - val deleteRetentionMs = getLong(LogConfig.DeleteRetentionMsProp) - val compactionLagMs = getLong(LogConfig.MinCompactionLagMsProp) - val maxCompactionLagMs = getLong(LogConfig.MaxCompactionLagMsProp) - val minCleanableRatio = getDouble(LogConfig.MinCleanableDirtyRatioProp) - val compact = getList(LogConfig.CleanupPolicyProp).asScala.map(_.toLowerCase(Locale.ROOT)).contains(LogConfig.Compact) - val delete = getList(LogConfig.CleanupPolicyProp).asScala.map(_.toLowerCase(Locale.ROOT)).contains(LogConfig.Delete) - val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp) - val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp) - val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase(Locale.ROOT) - val preallocate = getBoolean(LogConfig.PreAllocateEnableProp) - - /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ - @deprecated("3.0") - val messageFormatVersion = MetadataVersion.fromVersionString(getString(LogConfig.MessageFormatVersionProp)) - - val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp)) - val messageTimestampDifferenceMaxMs = getLong(LogConfig.MessageTimestampDifferenceMaxMsProp).longValue - val LeaderReplicationThrottledReplicas = getList(LogConfig.LeaderReplicationThrottledReplicasProp) - val FollowerReplicationThrottledReplicas = getList(LogConfig.FollowerReplicationThrottledReplicasProp) - val messageDownConversionEnable = getBoolean(LogConfig.MessageDownConversionEnableProp) - - class RemoteLogConfig { - val remoteStorageEnable = getBoolean(LogConfig.RemoteLogStorageEnableProp) - - val localRetentionMs: Long = { - val localLogRetentionMs = getLong(LogConfig.LocalLogRetentionMsProp) - - // -2 indicates to derive value from retentionMs property. - if(localLogRetentionMs == -2) retentionMs - else { - // Added validation here to check the effective value should not be more than RetentionMs. - if(localLogRetentionMs == -1 && retentionMs != -1) { - throw new ConfigException(LogConfig.LocalLogRetentionMsProp, localLogRetentionMs, s"Value must not be -1 as ${LogConfig.RetentionMsProp} value is set as $retentionMs.") - } - - if (localLogRetentionMs > retentionMs) { - throw new ConfigException(LogConfig.LocalLogRetentionMsProp, localLogRetentionMs, s"Value must not be more than property: ${LogConfig.RetentionMsProp} value.") - } - - localLogRetentionMs - } - } - - val localRetentionBytes: Long = { - val localLogRetentionBytes = getLong(LogConfig.LocalLogRetentionBytesProp) - - // -2 indicates to derive value from retentionSize property. - if(localLogRetentionBytes == -2) retentionSize - else { - // Added validation here to check the effective value should not be more than RetentionBytes. - if(localLogRetentionBytes == -1 && retentionSize != -1) { - throw new ConfigException(LogConfig.LocalLogRetentionBytesProp, localLogRetentionBytes, s"Value must not be -1 as ${LogConfig.RetentionBytesProp} value is set as $retentionSize.") - } - - if (localLogRetentionBytes > retentionSize) { - throw new ConfigException(LogConfig.LocalLogRetentionBytesProp, localLogRetentionBytes, s"Value must not be more than property: ${LogConfig.RetentionBytesProp} value.") - } - - localLogRetentionBytes - } - } - } - - private val _remoteLogConfig = new RemoteLogConfig() - def remoteLogConfig = _remoteLogConfig - - @nowarn("cat=deprecation") - def recordVersion = messageFormatVersion.highestSupportedRecordVersion - - def randomSegmentJitter: Long = - if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) - - def maxSegmentMs: Long = { - if (compact && maxCompactionLagMs > 0) math.min(maxCompactionLagMs, segmentMs) - else segmentMs - } - - def initFileSize: Int = { - if (preallocate) - segmentSize - else - 0 - } - - def overriddenConfigsAsLoggableString: String = { - val overriddenTopicProps = props.asScala.collect { - case (k: String, v) if overriddenConfigs.contains(k) => (k, v.asInstanceOf[AnyRef]) - } - ConfigUtils.configMapToRedactedString(overriddenTopicProps.asJava, configDef) - } -} - -object LogConfig { - - def main(args: Array[String]): Unit = { - println(configDef.toHtml(4, (config: String) => "topicconfigs_" + config)) - } - - val SegmentBytesProp = TopicConfig.SEGMENT_BYTES_CONFIG - val SegmentMsProp = TopicConfig.SEGMENT_MS_CONFIG - val SegmentJitterMsProp = TopicConfig.SEGMENT_JITTER_MS_CONFIG - val SegmentIndexBytesProp = TopicConfig.SEGMENT_INDEX_BYTES_CONFIG - val FlushMessagesProp = TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG - val FlushMsProp = TopicConfig.FLUSH_MS_CONFIG - val RetentionBytesProp = TopicConfig.RETENTION_BYTES_CONFIG - val RetentionMsProp = TopicConfig.RETENTION_MS_CONFIG - val RemoteLogStorageEnableProp = TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG - val LocalLogRetentionMsProp = TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG - val LocalLogRetentionBytesProp = TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG - val MaxMessageBytesProp = TopicConfig.MAX_MESSAGE_BYTES_CONFIG - val IndexIntervalBytesProp = TopicConfig.INDEX_INTERVAL_BYTES_CONFIG - val DeleteRetentionMsProp = TopicConfig.DELETE_RETENTION_MS_CONFIG - val MinCompactionLagMsProp = TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG - val MaxCompactionLagMsProp = TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG - val FileDeleteDelayMsProp = TopicConfig.FILE_DELETE_DELAY_MS_CONFIG - val MinCleanableDirtyRatioProp = TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG - val CleanupPolicyProp = TopicConfig.CLEANUP_POLICY_CONFIG - val Delete = TopicConfig.CLEANUP_POLICY_DELETE - val Compact = TopicConfig.CLEANUP_POLICY_COMPACT - val UncleanLeaderElectionEnableProp = TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG - val MinInSyncReplicasProp = TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG - val CompressionTypeProp = TopicConfig.COMPRESSION_TYPE_CONFIG - val PreAllocateEnableProp = TopicConfig.PREALLOCATE_CONFIG - - /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ - @deprecated("3.0") - val MessageFormatVersionProp = TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG - val MessageTimestampTypeProp = TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG - val MessageTimestampDifferenceMaxMsProp = TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG - val MessageDownConversionEnableProp = TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG - - // Leave these out of TopicConfig for now as they are replication quota configs - val LeaderReplicationThrottledReplicasProp = "leader.replication.throttled.replicas" - val FollowerReplicationThrottledReplicasProp = "follower.replication.throttled.replicas" - - val SegmentSizeDoc = TopicConfig.SEGMENT_BYTES_DOC - val SegmentMsDoc = TopicConfig.SEGMENT_MS_DOC - val SegmentJitterMsDoc = TopicConfig.SEGMENT_JITTER_MS_DOC - val MaxIndexSizeDoc = TopicConfig.SEGMENT_INDEX_BYTES_DOC - val FlushIntervalDoc = TopicConfig.FLUSH_MESSAGES_INTERVAL_DOC - val FlushMsDoc = TopicConfig.FLUSH_MS_DOC - val RetentionSizeDoc = TopicConfig.RETENTION_BYTES_DOC - val RetentionMsDoc = TopicConfig.RETENTION_MS_DOC - val RemoteLogStorageEnableDoc = TopicConfig.REMOTE_LOG_STORAGE_ENABLE_DOC - val LocalLogRetentionMsDoc = TopicConfig.LOCAL_LOG_RETENTION_MS_DOC - val LocalLogRetentionBytesDoc = TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC - val MaxMessageSizeDoc = TopicConfig.MAX_MESSAGE_BYTES_DOC - val IndexIntervalDoc = TopicConfig.INDEX_INTERVAL_BYTES_DOC - val FileDeleteDelayMsDoc = TopicConfig.FILE_DELETE_DELAY_MS_DOC - val DeleteRetentionMsDoc = TopicConfig.DELETE_RETENTION_MS_DOC - val MinCompactionLagMsDoc = TopicConfig.MIN_COMPACTION_LAG_MS_DOC - val MaxCompactionLagMsDoc = TopicConfig.MAX_COMPACTION_LAG_MS_DOC - val MinCleanableRatioDoc = TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_DOC - val CompactDoc = TopicConfig.CLEANUP_POLICY_DOC - val UncleanLeaderElectionEnableDoc = TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_DOC - val MinInSyncReplicasDoc = TopicConfig.MIN_IN_SYNC_REPLICAS_DOC - val CompressionTypeDoc = TopicConfig.COMPRESSION_TYPE_DOC - val PreAllocateEnableDoc = TopicConfig.PREALLOCATE_DOC - - /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ - @deprecated("3.0") - val MessageFormatVersionDoc = TopicConfig.MESSAGE_FORMAT_VERSION_DOC - - val MessageTimestampTypeDoc = TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC - val MessageTimestampDifferenceMaxMsDoc = TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC - val MessageDownConversionEnableDoc = TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC - - val LeaderReplicationThrottledReplicasDoc = "A list of replicas for which log replication should be throttled on " + - "the leader side. The list should describe a set of replicas in the form " + - "[PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively the wildcard '*' can be used to throttle " + - "all replicas for this topic." - val FollowerReplicationThrottledReplicasDoc = "A list of replicas for which log replication should be throttled on " + - "the follower side. The list should describe a set of " + "replicas in the form " + - "[PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively the wildcard '*' can be used to throttle " + - "all replicas for this topic." - - private[log] val ServerDefaultHeaderName = "Server Default Property" - - val configsWithNoServerDefaults: Set[String] = Set(RemoteLogStorageEnableProp, LocalLogRetentionMsProp, LocalLogRetentionBytesProp) - - // Package private for testing - private[log] class LogConfigDef(base: ConfigDef) extends ConfigDef(base) { - def this() = this(new ConfigDef) - - private final val serverDefaultConfigNames = mutable.Map[String, String]() - base match { - case b: LogConfigDef => serverDefaultConfigNames ++= b.serverDefaultConfigNames - case _ => - } - - def define(name: String, defType: ConfigDef.Type, defaultValue: Any, validator: Validator, - importance: ConfigDef.Importance, doc: String, serverDefaultConfigName: String): LogConfigDef = { - super.define(name, defType, defaultValue, validator, importance, doc) - serverDefaultConfigNames.put(name, serverDefaultConfigName) - this - } - - def define(name: String, defType: ConfigDef.Type, defaultValue: Any, importance: ConfigDef.Importance, - documentation: String, serverDefaultConfigName: String): LogConfigDef = { - super.define(name, defType, defaultValue, importance, documentation) - serverDefaultConfigNames.put(name, serverDefaultConfigName) - this - } - - def define(name: String, defType: ConfigDef.Type, importance: ConfigDef.Importance, documentation: String, - serverDefaultConfigName: String): LogConfigDef = { - super.define(name, defType, importance, documentation) - serverDefaultConfigNames.put(name, serverDefaultConfigName) - this - } - - override def headers = List("Name", "Description", "Type", "Default", "Valid Values", ServerDefaultHeaderName, - "Importance").asJava - - override def getConfigValue(key: ConfigKey, headerName: String): String = { - headerName match { - case ServerDefaultHeaderName => serverDefaultConfigNames.getOrElse(key.name, null) - case _ => super.getConfigValue(key, headerName) - } - } - - def serverConfigName(configName: String): Option[String] = serverDefaultConfigNames.get(configName) - } - - // Package private for testing, return a copy since it's a mutable global variable - private[kafka] def configDefCopy: LogConfigDef = new LogConfigDef(configDef) - - private val configDef: LogConfigDef = { - import org.apache.kafka.common.config.ConfigDef.Importance._ - import org.apache.kafka.common.config.ConfigDef.Range._ - import org.apache.kafka.common.config.ConfigDef.Type._ - import org.apache.kafka.common.config.ConfigDef.ValidString._ - - @nowarn("cat=deprecation") - val logConfigDef = new LogConfigDef() - .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, - SegmentSizeDoc, KafkaConfig.LogSegmentBytesProp) - .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(1), MEDIUM, SegmentMsDoc, - KafkaConfig.LogRollTimeMillisProp) - .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc, - KafkaConfig.LogRollTimeJitterMillisProp) - .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(4), MEDIUM, MaxIndexSizeDoc, - KafkaConfig.LogIndexSizeMaxBytesProp) - .define(FlushMessagesProp, LONG, Defaults.FlushInterval, atLeast(1), MEDIUM, FlushIntervalDoc, - KafkaConfig.LogFlushIntervalMessagesProp) - .define(FlushMsProp, LONG, Defaults.FlushMs, atLeast(0), MEDIUM, FlushMsDoc, - KafkaConfig.LogFlushIntervalMsProp) - // can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize - .define(RetentionBytesProp, LONG, Defaults.RetentionSize, MEDIUM, RetentionSizeDoc, - KafkaConfig.LogRetentionBytesProp) - // can be negative. See kafka.log.LogManager.cleanupExpiredSegments - .define(RetentionMsProp, LONG, Defaults.RetentionMs, atLeast(-1), MEDIUM, RetentionMsDoc, - KafkaConfig.LogRetentionTimeMillisProp) - .define(MaxMessageBytesProp, INT, Defaults.MaxMessageSize, atLeast(0), MEDIUM, MaxMessageSizeDoc, - KafkaConfig.MessageMaxBytesProp) - .define(IndexIntervalBytesProp, INT, Defaults.IndexInterval, atLeast(0), MEDIUM, IndexIntervalDoc, - KafkaConfig.LogIndexIntervalBytesProp) - .define(DeleteRetentionMsProp, LONG, Defaults.DeleteRetentionMs, atLeast(0), MEDIUM, - DeleteRetentionMsDoc, KafkaConfig.LogCleanerDeleteRetentionMsProp) - .define(MinCompactionLagMsProp, LONG, Defaults.MinCompactionLagMs, atLeast(0), MEDIUM, MinCompactionLagMsDoc, - KafkaConfig.LogCleanerMinCompactionLagMsProp) - .define(MaxCompactionLagMsProp, LONG, Defaults.MaxCompactionLagMs, atLeast(1), MEDIUM, MaxCompactionLagMsDoc, - KafkaConfig.LogCleanerMaxCompactionLagMsProp) - .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc, - KafkaConfig.LogDeleteDelayMsProp) - .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM, - MinCleanableRatioDoc, KafkaConfig.LogCleanerMinCleanRatioProp) - .define(CleanupPolicyProp, LIST, Defaults.CleanupPolicy, ValidList.in(LogConfig.Compact, LogConfig.Delete), MEDIUM, CompactDoc, - KafkaConfig.LogCleanupPolicyProp) - .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, - MEDIUM, UncleanLeaderElectionEnableDoc, KafkaConfig.UncleanLeaderElectionEnableProp) - .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc, - KafkaConfig.MinInSyncReplicasProp) - .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionType.names.asScala.toSeq:_*), - MEDIUM, CompressionTypeDoc, KafkaConfig.CompressionTypeProp) - .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable, MEDIUM, PreAllocateEnableDoc, - KafkaConfig.LogPreAllocateProp) - .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, new MetadataVersionValidator(), MEDIUM, MessageFormatVersionDoc, - KafkaConfig.LogMessageFormatVersionProp) - .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, MessageTimestampTypeDoc, - KafkaConfig.LogMessageTimestampTypeProp) - .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, - atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc, KafkaConfig.LogMessageTimestampDifferenceMaxMsProp) - .define(LeaderReplicationThrottledReplicasProp, LIST, Defaults.LeaderReplicationThrottledReplicas, ThrottledReplicaListValidator, MEDIUM, - LeaderReplicationThrottledReplicasDoc, LeaderReplicationThrottledReplicasProp) - .define(FollowerReplicationThrottledReplicasProp, LIST, Defaults.FollowerReplicationThrottledReplicas, ThrottledReplicaListValidator, MEDIUM, - FollowerReplicationThrottledReplicasDoc, FollowerReplicationThrottledReplicasProp) - .define(MessageDownConversionEnableProp, BOOLEAN, Defaults.MessageDownConversionEnable, LOW, - MessageDownConversionEnableDoc, KafkaConfig.LogMessageDownConversionEnableProp) - - // RemoteLogStorageEnableProp, LocalLogRetentionMsProp, LocalLogRetentionBytesProp do not have server default - // config names. - logConfigDef - // This define method is not overridden in LogConfig as these configs do not have server defaults yet. - .defineInternal(RemoteLogStorageEnableProp, BOOLEAN, Defaults.RemoteLogStorageEnable, null, MEDIUM, RemoteLogStorageEnableDoc) - .defineInternal(LocalLogRetentionMsProp, LONG, Defaults.LocalRetentionMs, atLeast(-2), MEDIUM, LocalLogRetentionMsDoc) - .defineInternal(LocalLogRetentionBytesProp, LONG, Defaults.LocalRetentionBytes, atLeast(-2), MEDIUM, LocalLogRetentionBytesDoc) - - logConfigDef - } - - def apply(): LogConfig = LogConfig(new Properties()) - - def configNames: Seq[String] = configDef.names.asScala.toSeq.sorted - - def serverConfigName(configName: String): Option[String] = configDef.serverConfigName(configName) - - def configType(configName: String): Option[ConfigDef.Type] = { - Option(configDef.configKeys.get(configName)).map(_.`type`) - } - - /** - * Create a log config instance using the given properties and defaults - */ - def fromProps(defaults: java.util.Map[_ <: Object, _ <: Object], overrides: Properties): LogConfig = { - val props = new Properties() - defaults.forEach { (k, v) => props.put(k, v) } - props ++= overrides - val overriddenKeys = overrides.keySet.asScala.map(_.asInstanceOf[String]).toSet - new LogConfig(props, overriddenKeys) - } - - /** - * Check that property names are valid - */ - def validateNames(props: Properties): Unit = { - val names = configNames - for(name <- props.asScala.keys) - if (!names.contains(name)) - throw new InvalidConfigurationException(s"Unknown topic config name: $name") - } - - private[kafka] def configKeys: Map[String, ConfigKey] = configDef.configKeys.asScala - - def validateValues(props: java.util.Map[_, _]): Unit = { - val minCompactionLag = props.get(MinCompactionLagMsProp).asInstanceOf[Long] - val maxCompactionLag = props.get(MaxCompactionLagMsProp).asInstanceOf[Long] - if (minCompactionLag > maxCompactionLag) { - throw new InvalidConfigurationException(s"conflict topic config setting $MinCompactionLagMsProp " + - s"($minCompactionLag) > $MaxCompactionLagMsProp ($maxCompactionLag)") - } - } - - /** - * Check that the given properties contain only valid log config names and that all values can be parsed and are valid - */ - def validate(props: Properties): Unit = { - validateNames(props) - val valueMaps = configDef.parse(props) - validateValues(valueMaps) - } - - /** - * Maps topic configurations to their equivalent broker configurations. - * - * Topics can be configured either by setting their dynamic topic configurations, or by - * setting equivalent broker configurations. For historical reasons, the equivalent broker - * configurations have different names. This table maps each topic configuration to its - * equivalent broker configurations. - * - * In some cases, the equivalent broker configurations must be transformed before they - * can be used. For example, log.roll.hours must be converted to milliseconds before it - * can be used as the value of segment.ms. - * - * The broker configurations will be used in the order specified here. In other words, if - * both the first and the second synonyms are configured, we will use only the value of - * the first synonym and ignore the second. - */ - @nowarn("cat=deprecation") - val AllTopicConfigSynonyms = Map( - SegmentBytesProp -> asList( - new ConfigSynonym(KafkaConfig.LogSegmentBytesProp)), - SegmentMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogRollTimeMillisProp), - new ConfigSynonym(KafkaConfig.LogRollTimeHoursProp, HOURS_TO_MILLISECONDS)), - SegmentJitterMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogRollTimeJitterMillisProp), - new ConfigSynonym(KafkaConfig.LogRollTimeJitterHoursProp, HOURS_TO_MILLISECONDS)), - SegmentIndexBytesProp -> asList( - new ConfigSynonym(KafkaConfig.LogIndexSizeMaxBytesProp)), - FlushMessagesProp -> asList( - new ConfigSynonym(KafkaConfig.LogFlushIntervalMessagesProp)), - FlushMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogFlushIntervalMsProp), - new ConfigSynonym(KafkaConfig.LogFlushSchedulerIntervalMsProp)), - RetentionBytesProp -> asList( - new ConfigSynonym(KafkaConfig.LogRetentionBytesProp)), - RetentionMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogRetentionTimeMillisProp), - new ConfigSynonym(KafkaConfig.LogRetentionTimeMinutesProp, MINUTES_TO_MILLISECONDS), - new ConfigSynonym(KafkaConfig.LogRetentionTimeHoursProp, HOURS_TO_MILLISECONDS)), - MaxMessageBytesProp -> asList( - new ConfigSynonym(KafkaConfig.MessageMaxBytesProp)), - IndexIntervalBytesProp -> asList( - new ConfigSynonym(KafkaConfig.LogIndexIntervalBytesProp)), - DeleteRetentionMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogCleanerDeleteRetentionMsProp)), - MinCompactionLagMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogCleanerMinCompactionLagMsProp)), - MaxCompactionLagMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogCleanerMaxCompactionLagMsProp)), - FileDeleteDelayMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogDeleteDelayMsProp)), - MinCleanableDirtyRatioProp -> asList( - new ConfigSynonym(KafkaConfig.LogCleanerMinCleanRatioProp)), - CleanupPolicyProp -> asList( - new ConfigSynonym(KafkaConfig.LogCleanupPolicyProp)), - UncleanLeaderElectionEnableProp -> asList( - new ConfigSynonym(KafkaConfig.UncleanLeaderElectionEnableProp)), - MinInSyncReplicasProp -> asList( - new ConfigSynonym(KafkaConfig.MinInSyncReplicasProp)), - CompressionTypeProp -> asList( - new ConfigSynonym(KafkaConfig.CompressionTypeProp)), - PreAllocateEnableProp -> asList( - new ConfigSynonym(KafkaConfig.LogPreAllocateProp)), - MessageFormatVersionProp -> asList( - new ConfigSynonym(KafkaConfig.LogMessageFormatVersionProp)), - MessageTimestampTypeProp -> asList( - new ConfigSynonym(KafkaConfig.LogMessageTimestampTypeProp)), - MessageTimestampDifferenceMaxMsProp -> asList( - new ConfigSynonym(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp)), - MessageDownConversionEnableProp -> asList( - new ConfigSynonym(KafkaConfig.LogMessageDownConversionEnableProp)), - ).asJava - - /** - * Map topic config to the broker config with highest priority. Some of these have additional synonyms - * that can be obtained using [[kafka.server.DynamicBrokerConfig#brokerConfigSynonyms]] - * or using [[AllTopicConfigSynonyms]] - */ - val TopicConfigSynonyms = AllTopicConfigSynonyms.asScala.map { - case (k, v) => k -> v.get(0).name() - } - - /** - * Copy the subset of properties that are relevant to Logs. The individual properties - * are listed here since the names are slightly different in each Config class... - */ - @nowarn("cat=deprecation") - def extractLogConfigMap( - kafkaConfig: KafkaConfig - ): java.util.Map[String, Object] = { - val logProps = new java.util.HashMap[String, Object]() - logProps.put(SegmentBytesProp, kafkaConfig.logSegmentBytes) - logProps.put(SegmentMsProp, kafkaConfig.logRollTimeMillis) - logProps.put(SegmentJitterMsProp, kafkaConfig.logRollTimeJitterMillis) - logProps.put(SegmentIndexBytesProp, kafkaConfig.logIndexSizeMaxBytes) - logProps.put(FlushMessagesProp, kafkaConfig.logFlushIntervalMessages) - logProps.put(FlushMsProp, kafkaConfig.logFlushIntervalMs) - logProps.put(RetentionBytesProp, kafkaConfig.logRetentionBytes) - logProps.put(RetentionMsProp, kafkaConfig.logRetentionTimeMillis: java.lang.Long) - logProps.put(MaxMessageBytesProp, kafkaConfig.messageMaxBytes) - logProps.put(IndexIntervalBytesProp, kafkaConfig.logIndexIntervalBytes) - logProps.put(DeleteRetentionMsProp, kafkaConfig.logCleanerDeleteRetentionMs) - logProps.put(MinCompactionLagMsProp, kafkaConfig.logCleanerMinCompactionLagMs) - logProps.put(MaxCompactionLagMsProp, kafkaConfig.logCleanerMaxCompactionLagMs) - logProps.put(FileDeleteDelayMsProp, kafkaConfig.logDeleteDelayMs) - logProps.put(MinCleanableDirtyRatioProp, kafkaConfig.logCleanerMinCleanRatio) - logProps.put(CleanupPolicyProp, kafkaConfig.logCleanupPolicy) - logProps.put(MinInSyncReplicasProp, kafkaConfig.minInSyncReplicas) - logProps.put(CompressionTypeProp, kafkaConfig.compressionType) - logProps.put(UncleanLeaderElectionEnableProp, kafkaConfig.uncleanLeaderElectionEnable) - logProps.put(PreAllocateEnableProp, kafkaConfig.logPreAllocateEnable) - logProps.put(MessageFormatVersionProp, kafkaConfig.logMessageFormatVersion.version) - logProps.put(MessageTimestampTypeProp, kafkaConfig.logMessageTimestampType.name) - logProps.put(MessageTimestampDifferenceMaxMsProp, kafkaConfig.logMessageTimestampDifferenceMaxMs: java.lang.Long) - logProps.put(MessageDownConversionEnableProp, kafkaConfig.logMessageDownConversionEnable: java.lang.Boolean) - logProps - } - - def shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion: MetadataVersion): Boolean = - interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1) - - class MessageFormatVersion(messageFormatVersionString: String, interBrokerProtocolVersionString: String) { - val messageFormatVersion = MetadataVersion.fromVersionString(messageFormatVersionString) - private val interBrokerProtocolVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString) - - def shouldIgnore: Boolean = shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion) - - def shouldWarn: Boolean = - interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1) && messageFormatVersion.highestSupportedRecordVersion.precedes(RecordVersion.V2) - - @nowarn("cat=deprecation") - def topicWarningMessage(topicName: String): String = { - s"Topic configuration ${LogConfig.MessageFormatVersionProp} with value `$messageFormatVersionString` is ignored " + - s"for `$topicName` 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." - } - - @nowarn("cat=deprecation") - def brokerWarningMessage: String = { - s"Broker configuration ${KafkaConfig.LogMessageFormatVersionProp} with value $messageFormatVersionString 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/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala index 048a993d1bbb2..3bfb3225b83fd 100644 --- a/core/src/main/scala/kafka/log/LogLoader.scala +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.InvalidOffsetException import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.snapshot.Snapshots -import org.apache.kafka.server.log.internals.{CorruptIndexException, LogDirFailureChannel, LogOffsetMetadata} +import org.apache.kafka.server.log.internals.{CorruptIndexException, LogConfig, LogDirFailureChannel, LogOffsetMetadata} import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import scala.collection.{Set, mutable} diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 2f6bf52f67e40..a82cf5d1bfddc 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -17,14 +17,12 @@ package kafka.log -import kafka.log.LogConfig.MessageFormatVersion import kafka.log.remote.RemoteIndexCache import java.io._ import java.nio.file.Files import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger - import kafka.metrics.KafkaMetricsGroup import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.metadata.ConfigRepository @@ -39,10 +37,12 @@ import scala.collection._ import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Success, Try} import kafka.utils.Implicits._ -import java.util.Properties +import org.apache.kafka.common.config.TopicConfig +import java.util.Properties import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{CleanerConfig, LogConfig, LogDirFailureChannel} +import org.apache.kafka.server.log.internals.LogConfig.MessageFormatVersion import scala.annotation.nowarn @@ -499,12 +499,12 @@ class LogManager(logDirs: Seq[File], var overrides = configRepository.topicConfig(topicName) // save memory by only including configs for topics with overrides if (!overrides.isEmpty) { - Option(overrides.getProperty(LogConfig.MessageFormatVersionProp)).foreach { versionString => + Option(overrides.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)).foreach { versionString => val messageFormatVersion = new MessageFormatVersion(versionString, interBrokerProtocolVersion.version) if (messageFormatVersion.shouldIgnore) { val copy = new Properties() copy.putAll(overrides) - copy.remove(LogConfig.MessageFormatVersionProp) + copy.remove(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG) overrides = copy if (messageFormatVersion.shouldWarn) @@ -1368,10 +1368,10 @@ object LogManager { brokerTopicStats: BrokerTopicStats, logDirFailureChannel: LogDirFailureChannel, keepPartitionMetadataFile: Boolean): LogManager = { - val defaultProps = LogConfig.extractLogConfigMap(config) + val defaultProps = config.extractLogConfigMap LogConfig.validateValues(defaultProps) - val defaultLogConfig = LogConfig(defaultProps) + val defaultLogConfig = new LogConfig(defaultProps) val cleanerConfig = LogCleaner.cleanerConfig(config) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala old mode 100755 new mode 100644 index 5620ecd552d89..d289df2ec47ee --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.FileRecords.{LogOffsetPosition, TimestampAndOffset} import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} -import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LazyIndex, LogOffsetMetadata, OffsetIndex, OffsetPosition, TimeIndex, TimestampOffset, TransactionIndex, TxnIndexSearchResult} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LazyIndex, LogConfig, LogOffsetMetadata, OffsetIndex, OffsetPosition, TimeIndex, TimestampOffset, TransactionIndex, TxnIndexSearchResult} import java.util.Optional import scala.jdk.CollectionConverters._ diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 20d2551e17d95..c982e10a3ef82 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -42,7 +42,7 @@ import org.apache.kafka.common.utils.{PrimitiveRef, Time, Utils} import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_0_IV0 -import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LogDirFailureChannel, LogOffsetMetadata, LogValidator} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogValidator} import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig import org.apache.kafka.server.record.BrokerCompressionType diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 28953ab18466d..b51785788b9cf 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -16,17 +16,17 @@ */ package kafka.raft -import kafka.log.{Defaults, LogConfig, LogOffsetSnapshot, ProducerStateManagerConfig, SnapshotGenerated, UnifiedLog} +import kafka.log.{LogOffsetSnapshot, ProducerStateManagerConfig, SnapshotGenerated, UnifiedLog} import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp} import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, KafkaConfig, RequestLocal} import kafka.utils.{CoreUtils, Logging, Scheduler} -import org.apache.kafka.common.config.AbstractConfig +import org.apache.kafka.common.config.{AbstractConfig, TopicConfig} import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.common.record.{ControlRecordUtils, MemoryRecords, Records} import org.apache.kafka.common.utils.{BufferSupplier, 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.log.internals.{AppendOrigin, LogDirFailureChannel} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig, LogDirFailureChannel} import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import java.io.File @@ -531,7 +531,7 @@ object MetadataLogConfig { config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp), maxBatchSizeInBytes, maxFetchSizeInBytes, - Defaults.FileDeleteDelayMs, + LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, config.getInt(KafkaConfig.NodeIdProp) ) } @@ -544,7 +544,7 @@ case class MetadataLogConfig(logSegmentBytes: Int, retentionMillis: Long, maxBatchSizeInBytes: Int, maxFetchSizeInBytes: Int, - fileDeleteDelayMs: Int, + fileDeleteDelayMs: Long, nodeId: Int) object KafkaMetadataLog extends Logging { @@ -557,16 +557,16 @@ object KafkaMetadataLog extends Logging { config: MetadataLogConfig ): KafkaMetadataLog = { val props = new Properties() - props.setProperty(LogConfig.MaxMessageBytesProp, config.maxBatchSizeInBytes.toString) - props.setProperty(LogConfig.SegmentBytesProp, config.logSegmentBytes.toString) - props.setProperty(LogConfig.SegmentMsProp, config.logSegmentMillis.toString) - props.setProperty(LogConfig.FileDeleteDelayMsProp, Defaults.FileDeleteDelayMs.toString) + 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) // Disable time and byte retention when deleting segments - props.setProperty(LogConfig.RetentionMsProp, "-1") - props.setProperty(LogConfig.RetentionBytesProp, "-1") + props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1") + props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1") LogConfig.validate(props) - val defaultLogConfig = LogConfig(props) + val defaultLogConfig = new LogConfig(props) if (config.logSegmentBytes < config.logSegmentMinBytes) { throw new InvalidConfigurationException( @@ -574,11 +574,11 @@ object KafkaMetadataLog extends Logging { ) } else if (defaultLogConfig.retentionMs >= 0) { throw new InvalidConfigurationException( - s"Cannot set ${LogConfig.RetentionMsProp} above -1: ${defaultLogConfig.retentionMs}." + s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${defaultLogConfig.retentionMs}." ) } else if (defaultLogConfig.retentionSize >= 0) { throw new InvalidConfigurationException( - s"Cannot set ${LogConfig.RetentionBytesProp} above -1: ${defaultLogConfig.retentionSize}." + s"Cannot set ${TopicConfig.RETENTION_BYTES_CONFIG} above -1: ${defaultLogConfig.retentionSize}." ) } diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 13be872878ede..f43a6c1785362 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -21,20 +21,20 @@ import java.net.{InetAddress, UnknownHostException} import java.util.Properties import DynamicConfig.Broker._ import kafka.controller.KafkaController -import kafka.log.LogConfig.MessageFormatVersion -import kafka.log.{LogConfig, LogManager} +import kafka.log.LogManager import kafka.network.ConnectionQuotas import kafka.security.CredentialProvider import kafka.server.Constants._ import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.Implicits._ import kafka.utils.Logging -import org.apache.kafka.common.config.ConfigDef.Validator -import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.internals.QuotaConfigs import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.metrics.Quota._ import org.apache.kafka.common.utils.Sanitizer +import org.apache.kafka.server.log.internals.{LogConfig, ThrottledReplicaListValidator} +import org.apache.kafka.server.log.internals.LogConfig.MessageFormatVersion import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -74,8 +74,8 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC debug(s"Removing $prop from broker ${kafkaConfig.brokerId} for topic $topic") } } - updateThrottledList(LogConfig.LeaderReplicationThrottledReplicasProp, quotas.leader) - updateThrottledList(LogConfig.FollowerReplicationThrottledReplicasProp, quotas.follower) + updateThrottledList(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, quotas.leader) + updateThrottledList(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, quotas.follower) if (Try(topicConfig.getProperty(KafkaConfig.UncleanLeaderElectionEnableProp).toBoolean).getOrElse(false)) { kafkaController.foreach(_.enableTopicUncleanLeaderElection(topic)) @@ -99,16 +99,16 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC @nowarn("cat=deprecation") def excludedConfigs(topic: String, topicConfig: Properties): Set[String] = { // Verify message format version - Option(topicConfig.getProperty(LogConfig.MessageFormatVersionProp)).flatMap { versionString => + Option(topicConfig.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)).flatMap { versionString => val messageFormatVersion = new MessageFormatVersion(versionString, kafkaConfig.interBrokerProtocolVersion.version) if (messageFormatVersion.shouldIgnore) { if (messageFormatVersion.shouldWarn) warn(messageFormatVersion.topicWarningMessage(topic)) - Some(LogConfig.MessageFormatVersionProp) + Some(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG) } else if (kafkaConfig.interBrokerProtocolVersion.isLessThan(messageFormatVersion.messageFormatVersion)) { - warn(s"Topic configuration ${LogConfig.MessageFormatVersionProp} is ignored for `$topic` because `$versionString` " + + warn(s"Topic configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG} is ignored for `$topic` because `$versionString` " + s"is higher than what is allowed by the inter-broker protocol version `${kafkaConfig.interBrokerProtocolVersionString}`") - Some(LogConfig.MessageFormatVersionProp) + Some(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG) } else None }.toSet @@ -225,25 +225,3 @@ class BrokerConfigHandler(private val brokerConfig: KafkaConfig, } } } - -object ThrottledReplicaListValidator extends Validator { - def ensureValidString(name: String, value: String): Unit = - ensureValid(name, value.split(",").map(_.trim).toSeq) - - override def ensureValid(name: String, value: Any): Unit = { - def check(proposed: Seq[Any]): Unit = { - if (!(proposed.forall(_.toString.trim.matches("([0-9]+:[0-9]+)?")) - || proposed.mkString.trim.equals("*"))) - throw new ConfigException(name, value, - s"$name must be the literal '*' or a list of replicas in the following format: [partitionId]:[brokerId],[partitionId]:[brokerId],...") - } - value match { - case scalaSeq: Seq[_] => check(scalaSeq) - case javaList: java.util.List[_] => check(javaList.asScala) - case _ => throw new ConfigException(name, value, s"$name must be a List but was ${value.getClass.getName}") - } - } - - override def toString: String = "[partitionId]:[brokerId],[partitionId]:[brokerId],..." - -} diff --git a/core/src/main/scala/kafka/server/ConfigHelper.scala b/core/src/main/scala/kafka/server/ConfigHelper.scala index 174d99823e5a0..721b82b375e11 100644 --- a/core/src/main/scala/kafka/server/ConfigHelper.scala +++ b/core/src/main/scala/kafka/server/ConfigHelper.scala @@ -18,8 +18,6 @@ package kafka.server import java.util.{Collections, Properties} - -import kafka.log.LogConfig import kafka.server.metadata.ConfigRepository import kafka.utils.{Log4jController, Logging} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigResource} @@ -30,8 +28,11 @@ import org.apache.kafka.common.message.DescribeConfigsResponseData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ApiError, DescribeConfigsResponse} import org.apache.kafka.common.requests.DescribeConfigsResponse.ConfigSource +import org.apache.kafka.server.config.ServerTopicConfigSynonyms +import org.apache.kafka.server.log.internals.LogConfig import scala.collection.{Map, mutable} +import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ class ConfigHelper(metadataCache: MetadataCache, config: KafkaConfig, configRepository: ConfigRepository) extends Logging { @@ -66,7 +67,7 @@ class ConfigHelper(metadataCache: MetadataCache, config: KafkaConfig, configRepo Topic.validate(topic) if (metadataCache.contains(topic)) { val topicProps = configRepository.topicConfig(topic) - val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(config), topicProps) + val logConfig = LogConfig.fromProps(config.extractLogConfigMap, topicProps) createResponseConfig(allConfigs(logConfig), createTopicConfigEntry(logConfig, topicProps, includeSynonyms, includeDocumentation)) } else { new DescribeConfigsResponseData.DescribeConfigsResult().setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) @@ -117,11 +118,11 @@ class ConfigHelper(metadataCache: MetadataCache, config: KafkaConfig, configRepo def createTopicConfigEntry(logConfig: LogConfig, topicProps: Properties, includeSynonyms: Boolean, includeDocumentation: Boolean) (name: String, value: Any): DescribeConfigsResponseData.DescribeConfigsResourceResult = { - val configEntryType = LogConfig.configType(name) + val configEntryType = LogConfig.configType(name).asScala val isSensitive = KafkaConfig.maybeSensitive(configEntryType) val valueAsString = if (isSensitive) null else ConfigDef.convertToString(value, configEntryType.orNull) val allSynonyms = { - val list = LogConfig.TopicConfigSynonyms.get(name) + val list = Option(ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.get(name)) .map(s => configSynonyms(s, brokerSynonyms(s), isSensitive)) .getOrElse(List.empty) if (!topicProps.containsKey(name)) diff --git a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala index a5dc78d388788..dec95ca359da2 100644 --- a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala +++ b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala @@ -19,13 +19,12 @@ package kafka.server import java.util import java.util.Properties - -import kafka.log.LogConfig import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, TOPIC} import org.apache.kafka.controller.ConfigurationValidator import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException} import org.apache.kafka.common.internals.Topic +import org.apache.kafka.server.log.internals.LogConfig import scala.collection.mutable @@ -112,4 +111,4 @@ class ControllerConfigurationValidator extends ConfigurationValidator { case _ => throwExceptionForUnknownResourceType(resource) } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 77f9ba546fdd6..3ae18bb001b88 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -35,11 +35,11 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.utils.LogContext import org.apache.kafka.common.{ClusterResource, Endpoint} import org.apache.kafka.controller.{Controller, QuorumController, QuorumFeatures} -import org.apache.kafka.metadata.KafkaConfigSchema import org.apache.kafka.raft.RaftConfig import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.metadata.KafkaConfigSchema import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.metadata.bootstrap.BootstrapMetadata import org.apache.kafka.server.metrics.KafkaYammerMetrics diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 38f7722d34378..a0efb0649baa4 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -22,7 +22,7 @@ import java.util.{Collections, Properties} import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.cluster.EndPoint -import kafka.log.{LogCleaner, LogConfig, LogManager, ProducerStateManagerConfig} +import kafka.log.{LogCleaner, LogManager, ProducerStateManagerConfig} import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.server.DynamicBrokerConfig._ import kafka.utils.{CoreUtils, Logging, PasswordEncoder} @@ -35,6 +35,8 @@ import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} import org.apache.kafka.common.security.authenticator.LoginManager import org.apache.kafka.common.utils.{ConfigUtils, Utils} +import org.apache.kafka.server.config.ServerTopicConfigSynonyms +import org.apache.kafka.server.log.internals.LogConfig import scala.annotation.nowarn import scala.collection._ @@ -638,8 +640,8 @@ object DynamicLogConfig { @nowarn("cat=deprecation") val ExcludedConfigs = Set(KafkaConfig.LogMessageFormatVersionProp) - val ReconfigurableConfigs = LogConfig.TopicConfigSynonyms.values.toSet -- ExcludedConfigs - val KafkaConfigToLogConfigName = LogConfig.TopicConfigSynonyms.map { case (k, v) => (v, k) } + val ReconfigurableConfigs = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.values.asScala.toSet -- ExcludedConfigs + val KafkaConfigToLogConfigName = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) } } class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends BrokerReconfigurable with Logging { @@ -663,7 +665,7 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok log.config.overriddenConfigs.contains(k) } - val logConfig = LogConfig(props.asJava, log.config.overriddenConfigs) + val logConfig = new LogConfig(props.asJava, log.config.overriddenConfigs) log.updateConfig(logConfig) } } @@ -683,7 +685,7 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok } } - logManager.reconfigureDefaultLogConfig(LogConfig(newBrokerDefaults)) + logManager.reconfigureDefaultLogConfig(new LogConfig(newBrokerDefaults)) updateLogsConfig(newBrokerDefaults.asScala) diff --git a/core/src/main/scala/kafka/server/DynamicConfig.scala b/core/src/main/scala/kafka/server/DynamicConfig.scala index e3479e3641a55..08e25dd9adaed 100644 --- a/core/src/main/scala/kafka/server/DynamicConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicConfig.scala @@ -19,12 +19,11 @@ package kafka.server import java.net.{InetAddress, UnknownHostException} import java.util.Properties - -import kafka.log.LogConfig import org.apache.kafka.common.config.ConfigDef import org.apache.kafka.common.config.ConfigDef.Importance._ import org.apache.kafka.common.config.ConfigDef.Range._ import org.apache.kafka.common.config.ConfigDef.Type._ +import org.apache.kafka.server.log.internals.LogConfig import scala.jdk.CollectionConverters._ @@ -45,10 +44,10 @@ object DynamicConfig { // Documentation val LeaderReplicationThrottledRateDoc = "A long representing the upper bound (bytes/sec) on replication traffic for leaders enumerated in the " + - s"property ${LogConfig.LeaderReplicationThrottledReplicasProp} (for each topic). This property can be only set dynamically. It is suggested that the " + + s"property ${LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG} (for each topic). This property can be only set dynamically. It is suggested that the " + s"limit be kept above 1MB/s for accurate behaviour." val FollowerReplicationThrottledRateDoc = "A long representing the upper bound (bytes/sec) on replication traffic for followers enumerated in the " + - s"property ${LogConfig.FollowerReplicationThrottledReplicasProp} (for each topic). This property can be only set dynamically. It is suggested that the " + + s"property ${LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG} (for each topic). This property can be only set dynamically. It is suggested that the " + s"limit be kept above 1MB/s for accurate behaviour." val ReplicaAlterLogDirsIoMaxBytesPerSecondDoc = "A long representing the upper bound (bytes/sec) on disk IO used for moving replica between log directories on the same broker. " + s"This property can be only set dynamically. It is suggested that the limit be kept above 1MB/s for accurate behaviour." diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 13839bb88ba83..5903a495a4747 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -23,8 +23,6 @@ import java.util.{Collections, Locale, Properties} import kafka.cluster.EndPoint import kafka.coordinator.group.OffsetConfig import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager} -import kafka.log.LogConfig -import kafka.log.LogConfig.MessageFormatVersion import kafka.security.authorizer.AuthorizerUtils import kafka.server.KafkaConfig.{ControllerListenerNamesProp, ListenerSecurityProtocolMapProp} import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole, ProcessRole} @@ -48,11 +46,15 @@ import org.apache.kafka.raft.RaftConfig 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.ServerTopicConfigSynonyms +import org.apache.kafka.server.log.internals.LogConfig +import org.apache.kafka.server.log.internals.LogConfig.MessageFormatVersion import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.record.BrokerCompressionType import org.apache.zookeeper.client.ZKClientConfig import scala.annotation.nowarn +import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} @@ -71,7 +73,6 @@ object Defaults { val BrokerIdGenerationEnable = true val MaxReservedBrokerId = 1000 val BrokerId = -1 - val MessageMaxBytes = 1024 * 1024 + Records.LOG_OVERHEAD val NumNetworkThreads = 3 val NumIoThreads = 8 val BackgroundThreads = 10 @@ -115,47 +116,18 @@ object Defaults { /** ********* Log Configuration ***********/ val NumPartitions = 1 val LogDir = "/tmp/kafka-logs" - val LogSegmentBytes = 1 * 1024 * 1024 * 1024 - val LogRollHours = 24 * 7 - val LogRollJitterHours = 0 - val LogRetentionHours = 24 * 7 - - val LogRetentionBytes = -1L val LogCleanupIntervalMs = 5 * 60 * 1000L - val Delete = "delete" - val Compact = "compact" - val LogCleanupPolicy = Delete val LogCleanerThreads = 1 val LogCleanerIoMaxBytesPerSecond = Double.MaxValue val LogCleanerDedupeBufferSize = 128 * 1024 * 1024L val LogCleanerIoBufferSize = 512 * 1024 val LogCleanerDedupeBufferLoadFactor = 0.9d val LogCleanerBackoffMs = 15 * 1000 - val LogCleanerMinCleanRatio = 0.5d val LogCleanerEnable = true - val LogCleanerDeleteRetentionMs = 24 * 60 * 60 * 1000L - val LogCleanerMinCompactionLagMs = 0L - val LogCleanerMaxCompactionLagMs = Long.MaxValue - val LogIndexSizeMaxBytes = 10 * 1024 * 1024 - val LogIndexIntervalBytes = 4096 - val LogFlushIntervalMessages = Long.MaxValue - val LogDeleteDelayMs = 60000 - val LogFlushSchedulerIntervalMs = Long.MaxValue val LogFlushOffsetCheckpointIntervalMs = 60000 val LogFlushStartOffsetCheckpointIntervalMs = 60000 - val LogPreAllocateEnable = false - - /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ - @deprecated("3.0") - val LogMessageFormatVersion = IBP_3_0_IV1.version - - val LogMessageTimestampType = "CreateTime" - val LogMessageTimestampDifferenceMaxMs = Long.MaxValue val NumRecoveryThreadsPerDataDir = 1 val AutoCreateTopicsEnable = true - val MinInSyncReplicas = 1 - val MessageDownConversionEnable = true - val CompressionType = BrokerCompressionType.PRODUCER.name /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMs = RequestTimeoutMs @@ -176,7 +148,6 @@ object Defaults { val AutoLeaderRebalanceEnable = true val LeaderImbalancePerBrokerPercentage = 10 val LeaderImbalanceCheckIntervalSeconds = 300 - val UncleanLeaderElectionEnable = false val InterBrokerSecurityProtocol = SecurityProtocol.PLAINTEXT.toString val InterBrokerProtocolVersion = MetadataVersion.latest.version @@ -434,54 +405,54 @@ object KafkaConfig { val NumPartitionsProp = "num.partitions" val LogDirsProp = LogConfigPrefix + "dirs" val LogDirProp = LogConfigPrefix + "dir" - val LogSegmentBytesProp = LogConfigPrefix + "segment.bytes" + val LogSegmentBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG) - val LogRollTimeMillisProp = LogConfigPrefix + "roll.ms" + val LogRollTimeMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG) val LogRollTimeHoursProp = LogConfigPrefix + "roll.hours" - val LogRollTimeJitterMillisProp = LogConfigPrefix + "roll.jitter.ms" + val LogRollTimeJitterMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_JITTER_MS_CONFIG) val LogRollTimeJitterHoursProp = LogConfigPrefix + "roll.jitter.hours" - val LogRetentionTimeMillisProp = LogConfigPrefix + "retention.ms" + val LogRetentionTimeMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_MS_CONFIG) val LogRetentionTimeMinutesProp = LogConfigPrefix + "retention.minutes" val LogRetentionTimeHoursProp = LogConfigPrefix + "retention.hours" - val LogRetentionBytesProp = LogConfigPrefix + "retention.bytes" + val LogRetentionBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_BYTES_CONFIG) val LogCleanupIntervalMsProp = LogConfigPrefix + "retention.check.interval.ms" - val LogCleanupPolicyProp = LogConfigPrefix + "cleanup.policy" + val LogCleanupPolicyProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG) val LogCleanerThreadsProp = LogConfigPrefix + "cleaner.threads" val LogCleanerIoMaxBytesPerSecondProp = LogConfigPrefix + "cleaner.io.max.bytes.per.second" val LogCleanerDedupeBufferSizeProp = LogConfigPrefix + "cleaner.dedupe.buffer.size" val LogCleanerIoBufferSizeProp = LogConfigPrefix + "cleaner.io.buffer.size" val LogCleanerDedupeBufferLoadFactorProp = LogConfigPrefix + "cleaner.io.buffer.load.factor" val LogCleanerBackoffMsProp = LogConfigPrefix + "cleaner.backoff.ms" - val LogCleanerMinCleanRatioProp = LogConfigPrefix + "cleaner.min.cleanable.ratio" + val LogCleanerMinCleanRatioProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG) val LogCleanerEnableProp = LogConfigPrefix + "cleaner.enable" - val LogCleanerDeleteRetentionMsProp = LogConfigPrefix + "cleaner.delete.retention.ms" - val LogCleanerMinCompactionLagMsProp = LogConfigPrefix + "cleaner.min.compaction.lag.ms" - val LogCleanerMaxCompactionLagMsProp = LogConfigPrefix + "cleaner.max.compaction.lag.ms" - val LogIndexSizeMaxBytesProp = LogConfigPrefix + "index.size.max.bytes" - val LogIndexIntervalBytesProp = LogConfigPrefix + "index.interval.bytes" - val LogFlushIntervalMessagesProp = LogConfigPrefix + "flush.interval.messages" - val LogDeleteDelayMsProp = LogConfigPrefix + "segment.delete.delay.ms" + val LogCleanerDeleteRetentionMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.DELETE_RETENTION_MS_CONFIG) + val LogCleanerMinCompactionLagMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG) + val LogCleanerMaxCompactionLagMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MAX_COMPACTION_LAG_MS_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 = LogConfigPrefix + "flush.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 = LogConfigPrefix + "preallocate" + val LogPreAllocateProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.PREALLOCATE_CONFIG) /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ @deprecated("3.0") - val LogMessageFormatVersionProp = LogConfigPrefix + "message.format.version" + val LogMessageFormatVersionProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG) - val LogMessageTimestampTypeProp = LogConfigPrefix + "message.timestamp.type" - val LogMessageTimestampDifferenceMaxMsProp = LogConfigPrefix + "message.timestamp.difference.max.ms" + val LogMessageTimestampTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG) + val LogMessageTimestampDifferenceMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG) val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" val AutoCreateTopicsEnableProp = "auto.create.topics.enable" - val MinInSyncReplicasProp = "min.insync.replicas" + 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 = LogConfigPrefix + "message.downconversion.enable" + val LogMessageDownConversionEnableProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG) /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" val DefaultReplicationFactorProp = "default.replication.factor" @@ -501,7 +472,7 @@ object KafkaConfig { val AutoLeaderRebalanceEnableProp = "auto.leader.rebalance.enable" val LeaderImbalancePerBrokerPercentageProp = "leader.imbalance.per.broker.percentage" val LeaderImbalanceCheckIntervalSecondsProp = "leader.imbalance.check.interval.seconds" - val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" + val UncleanLeaderElectionEnableProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG) val InterBrokerSecurityProtocolProp = "security.inter.broker.protocol" val InterBrokerProtocolVersionProp = "inter.broker.protocol.version" val InterBrokerListenerNameProp = "inter.broker.listener.name" @@ -556,7 +527,7 @@ object KafkaConfig { val ClientQuotaCallbackClassProp = "client.quota.callback.class" val DeleteTopicEnableProp = "delete.topic.enable" - val CompressionTypeProp = "compression.type" + val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG) /** ********* Kafka Metrics Configuration ***********/ val MetricSampleWindowMsProp = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG @@ -1148,7 +1119,7 @@ object KafkaConfig { .define(BrokerIdGenerationEnableProp, BOOLEAN, Defaults.BrokerIdGenerationEnable, MEDIUM, BrokerIdGenerationEnableDoc) .define(MaxReservedBrokerIdProp, INT, Defaults.MaxReservedBrokerId, atLeast(0), MEDIUM, MaxReservedBrokerIdDoc) .define(BrokerIdProp, INT, Defaults.BrokerId, HIGH, BrokerIdDoc) - .define(MessageMaxBytesProp, INT, Defaults.MessageMaxBytes, atLeast(0), HIGH, MessageMaxBytesDoc) + .define(MessageMaxBytesProp, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MessageMaxBytesDoc) .define(NumNetworkThreadsProp, INT, Defaults.NumNetworkThreads, atLeast(1), HIGH, NumNetworkThreadsDoc) .define(NumIoThreadsProp, INT, Defaults.NumIoThreads, atLeast(1), HIGH, NumIoThreadsDoc) .define(NumReplicaAlterLogDirsThreadsProp, INT, null, HIGH, NumReplicaAlterLogDirsThreadsDoc) @@ -1176,11 +1147,11 @@ object KafkaConfig { .define(ControllerListenerNamesProp, STRING, null, null, HIGH, ControllerListenerNamesDoc) .define(SaslMechanismControllerProtocolProp, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SaslMechanismControllerProtocolDoc) .define(MetadataLogDirProp, STRING, null, null, HIGH, MetadataLogDirDoc) - .define(MetadataLogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentBytesDoc) + .define(MetadataLogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentBytesDoc) .defineInternal(MetadataLogSegmentMinBytesProp, INT, 8 * 1024 * 1024, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentMinBytesDoc) - .define(MetadataLogSegmentMillisProp, LONG, Defaults.LogRollHours * 60 * 60 * 1000L, null, HIGH, MetadataLogSegmentMillisDoc) + .define(MetadataLogSegmentMillisProp, LONG, LogConfig.DEFAULT_SEGMENT_MS, null, HIGH, MetadataLogSegmentMillisDoc) .define(MetadataMaxRetentionBytesProp, LONG, Defaults.MetadataMaxRetentionBytes, null, HIGH, MetadataMaxRetentionBytesDoc) - .define(MetadataMaxRetentionMillisProp, LONG, Defaults.LogRetentionHours * 60 * 60 * 1000L, null, HIGH, MetadataMaxRetentionMillisDoc) + .define(MetadataMaxRetentionMillisProp, LONG, LogConfig.DEFAULT_RETENTION_MS, null, HIGH, MetadataMaxRetentionMillisDoc) .define(MetadataMaxIdleIntervalMsProp, INT, Defaults.MetadataMaxIdleIntervalMs, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc) .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration") @@ -1211,50 +1182,50 @@ object KafkaConfig { .define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc) .define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc) .define(LogDirsProp, STRING, null, HIGH, LogDirsDoc) - .define(LogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, LogSegmentBytesDoc) + .define(LogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, LogSegmentBytesDoc) .define(LogRollTimeMillisProp, LONG, null, HIGH, LogRollTimeMillisDoc) - .define(LogRollTimeHoursProp, INT, Defaults.LogRollHours, atLeast(1), HIGH, LogRollTimeHoursDoc) + .define(LogRollTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_MS).toInt, atLeast(1), HIGH, LogRollTimeHoursDoc) .define(LogRollTimeJitterMillisProp, LONG, null, HIGH, LogRollTimeJitterMillisDoc) - .define(LogRollTimeJitterHoursProp, INT, Defaults.LogRollJitterHours, atLeast(0), HIGH, LogRollTimeJitterHoursDoc) + .define(LogRollTimeJitterHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_JITTER_MS).toInt, atLeast(0), HIGH, LogRollTimeJitterHoursDoc) .define(LogRetentionTimeMillisProp, LONG, null, HIGH, LogRetentionTimeMillisDoc) .define(LogRetentionTimeMinutesProp, INT, null, HIGH, LogRetentionTimeMinsDoc) - .define(LogRetentionTimeHoursProp, INT, Defaults.LogRetentionHours, HIGH, LogRetentionTimeHoursDoc) + .define(LogRetentionTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_RETENTION_MS).toInt, HIGH, LogRetentionTimeHoursDoc) - .define(LogRetentionBytesProp, LONG, Defaults.LogRetentionBytes, HIGH, LogRetentionBytesDoc) + .define(LogRetentionBytesProp, LONG, LogConfig.DEFAULT_RETENTION_BYTES, HIGH, LogRetentionBytesDoc) .define(LogCleanupIntervalMsProp, LONG, Defaults.LogCleanupIntervalMs, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc) - .define(LogCleanupPolicyProp, LIST, Defaults.LogCleanupPolicy, ValidList.in(Defaults.Compact, Defaults.Delete), MEDIUM, LogCleanupPolicyDoc) + .define(LogCleanupPolicyProp, LIST, LogConfig.DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, LogCleanupPolicyDoc) .define(LogCleanerThreadsProp, INT, Defaults.LogCleanerThreads, atLeast(0), MEDIUM, LogCleanerThreadsDoc) .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LogCleanerIoMaxBytesPerSecond, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc) .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LogCleanerDedupeBufferSize, MEDIUM, LogCleanerDedupeBufferSizeDoc) .define(LogCleanerIoBufferSizeProp, INT, Defaults.LogCleanerIoBufferSize, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LogCleanerDedupeBufferLoadFactor, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) .define(LogCleanerBackoffMsProp, LONG, Defaults.LogCleanerBackoffMs, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) - .define(LogCleanerMinCleanRatioProp, DOUBLE, Defaults.LogCleanerMinCleanRatio, between(0, 1), MEDIUM, LogCleanerMinCleanRatioDoc) + .define(LogCleanerMinCleanRatioProp, DOUBLE, LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM, LogCleanerMinCleanRatioDoc) .define(LogCleanerEnableProp, BOOLEAN, Defaults.LogCleanerEnable, MEDIUM, LogCleanerEnableDoc) - .define(LogCleanerDeleteRetentionMsProp, LONG, Defaults.LogCleanerDeleteRetentionMs, atLeast(0), MEDIUM, LogCleanerDeleteRetentionMsDoc) - .define(LogCleanerMinCompactionLagMsProp, LONG, Defaults.LogCleanerMinCompactionLagMs, atLeast(0), MEDIUM, LogCleanerMinCompactionLagMsDoc) - .define(LogCleanerMaxCompactionLagMsProp, LONG, Defaults.LogCleanerMaxCompactionLagMs, atLeast(1), MEDIUM, LogCleanerMaxCompactionLagMsDoc) - .define(LogIndexSizeMaxBytesProp, INT, Defaults.LogIndexSizeMaxBytes, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc) - .define(LogIndexIntervalBytesProp, INT, Defaults.LogIndexIntervalBytes, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc) - .define(LogFlushIntervalMessagesProp, LONG, Defaults.LogFlushIntervalMessages, atLeast(1), HIGH, LogFlushIntervalMessagesDoc) - .define(LogDeleteDelayMsProp, LONG, Defaults.LogDeleteDelayMs, atLeast(0), HIGH, LogDeleteDelayMsDoc) - .define(LogFlushSchedulerIntervalMsProp, LONG, Defaults.LogFlushSchedulerIntervalMs, HIGH, LogFlushSchedulerIntervalMsDoc) + .define(LogCleanerDeleteRetentionMsProp, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, LogCleanerDeleteRetentionMsDoc) + .define(LogCleanerMinCompactionLagMsProp, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, LogCleanerMinCompactionLagMsDoc) + .define(LogCleanerMaxCompactionLagMsProp, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, LogCleanerMaxCompactionLagMsDoc) + .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.LogFlushOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) .define(LogFlushStartOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushStartOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushStartOffsetCheckpointIntervalMsDoc) - .define(LogPreAllocateProp, BOOLEAN, Defaults.LogPreAllocateEnable, MEDIUM, LogPreAllocateEnableDoc) + .define(LogPreAllocateProp, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, LogPreAllocateEnableDoc) .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) - .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) - .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, new MetadataVersionValidator(), MEDIUM, LogMessageFormatVersionDoc) - .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) - .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) + .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(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc) .define(AlterConfigPolicyClassNameProp, CLASS, null, LOW, AlterConfigPolicyClassNameDoc) - .define(LogMessageDownConversionEnableProp, BOOLEAN, Defaults.MessageDownConversionEnable, LOW, LogMessageDownConversionEnableDoc) + .define(LogMessageDownConversionEnableProp, BOOLEAN, LogConfig.DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, LogMessageDownConversionEnableDoc) /** ********* Replication configuration ***********/ .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) @@ -1275,7 +1246,7 @@ object KafkaConfig { .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, atLeast(1), HIGH, LeaderImbalanceCheckIntervalSecondsDoc) - .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) + .define(UncleanLeaderElectionEnableProp, BOOLEAN, LogConfig.DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE, HIGH, UncleanLeaderElectionEnableDoc) .define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, InterBrokerSecurityProtocolDoc) .define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, new MetadataVersionValidator(), MEDIUM, InterBrokerProtocolVersionDoc) .define(InterBrokerListenerNameProp, STRING, null, MEDIUM, InterBrokerListenerNameDoc) @@ -1304,7 +1275,7 @@ object KafkaConfig { .define(OffsetCommitTimeoutMsProp, INT, Defaults.OffsetCommitTimeoutMs, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc) .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) - .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) + .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) /** ********* Transaction management configuration ***********/ .define(TransactionalIdExpirationMsProp, INT, Defaults.TransactionalIdExpirationMs, atLeast(1), HIGH, TransactionalIdExpirationMsDoc) @@ -1495,7 +1466,7 @@ object KafkaConfig { def loggableValue(resourceType: ConfigResource.Type, name: String, value: String): String = { val maybeSensitive = resourceType match { case ConfigResource.Type.BROKER => KafkaConfig.maybeSensitive(KafkaConfig.configType(name)) - case ConfigResource.Type.TOPIC => KafkaConfig.maybeSensitive(LogConfig.configType(name)) + case ConfigResource.Type.TOPIC => KafkaConfig.maybeSensitive(LogConfig.configType(name).asScala) case ConfigResource.Type.BROKER_LOGGER => false case _ => true } @@ -1801,7 +1772,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami @deprecated("3.0") lazy val logMessageFormatVersion = if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion)) - MetadataVersion.fromVersionString(Defaults.LogMessageFormatVersion) + MetadataVersion.fromVersionString(LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION) else MetadataVersion.fromVersionString(logMessageFormatVersionString) def logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp)) @@ -2262,7 +2233,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val messageFormatVersion = new MessageFormatVersion(logMessageFormatVersionString, interBrokerProtocolVersionString) if (messageFormatVersion.shouldWarn) - warn(messageFormatVersion.brokerWarningMessage) + warn(createBrokerWarningMessage) val recordVersion = logMessageFormatVersion.highestSupportedRecordVersion require(interBrokerProtocolVersion.highestSupportedRecordVersion().value >= recordVersion.value, @@ -2301,4 +2272,45 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami require(classOf[KafkaPrincipalSerde].isAssignableFrom(principalBuilderClass), s"${KafkaConfig.PrincipalBuilderClassProp} must implement KafkaPrincipalSerde") } + + /** + * Copy the subset of properties that are relevant to Logs. The individual properties + * are listed here since the names are slightly different in each Config class... + */ + @nowarn("cat=deprecation") + def extractLogConfigMap: java.util.Map[String, Object] = { + val logProps = new java.util.HashMap[String, Object]() + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, logSegmentBytes) + logProps.put(TopicConfig.SEGMENT_MS_CONFIG, logRollTimeMillis) + logProps.put(TopicConfig.SEGMENT_JITTER_MS_CONFIG, logRollTimeJitterMillis) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, logIndexSizeMaxBytes) + logProps.put(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, logFlushIntervalMessages) + logProps.put(TopicConfig.FLUSH_MS_CONFIG, logFlushIntervalMs) + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, logRetentionBytes) + logProps.put(TopicConfig.RETENTION_MS_CONFIG, logRetentionTimeMillis: java.lang.Long) + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, messageMaxBytes) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, logIndexIntervalBytes) + logProps.put(TopicConfig.DELETE_RETENTION_MS_CONFIG, logCleanerDeleteRetentionMs) + logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, logCleanerMinCompactionLagMs) + logProps.put(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, logCleanerMaxCompactionLagMs) + logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, logDeleteDelayMs) + logProps.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, logCleanerMinCleanRatio) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, logCleanupPolicy) + logProps.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, minInSyncReplicas) + logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType) + logProps.put(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, uncleanLeaderElectionEnable) + logProps.put(TopicConfig.PREALLOCATE_CONFIG, logPreAllocateEnable) + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version) + logProps.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, logMessageTimestampType.name) + logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, logMessageTimestampDifferenceMaxMs: java.lang.Long) + logProps.put(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, logMessageDownConversionEnable: java.lang.Boolean) + logProps + } + + @nowarn("cat=deprecation") + private def createBrokerWarningMessage: String = { + s"Broker configuration ${KafkaConfig.LogMessageFormatVersionProp} 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 1c5f3f92648ce..9cb95e39166ef 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -19,7 +19,7 @@ package kafka.server import java.io.File import java.util.concurrent.CompletableFuture import kafka.common.InconsistentNodeIdException -import kafka.log.{LogConfig, UnifiedLog} +import kafka.log.UnifiedLog import kafka.metrics.KafkaMetricsReporter import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} @@ -27,9 +27,11 @@ import org.apache.kafka.common.config.{ConfigDef, ConfigResource} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.utils.{AppInfoParser, Time} import org.apache.kafka.common.{KafkaException, Uuid} -import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} import org.apache.kafka.metadata.KafkaConfigSchema +import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} import org.apache.kafka.raft.RaftConfig +import org.apache.kafka.server.config.ServerTopicConfigSynonyms +import org.apache.kafka.server.log.internals.LogConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import java.util.Optional @@ -172,5 +174,5 @@ object KafkaRaftServer { val configSchema = new KafkaConfigSchema(Map( ConfigResource.Type.BROKER -> new ConfigDef(KafkaConfig.configDef), ConfigResource.Type.TOPIC -> LogConfig.configDefCopy, - ).asJava, LogConfig.AllTopicConfigSynonyms) + ).asJava, ServerTopicConfigSynonyms.ALL_TOPIC_CONFIG_SYNONYMS) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index f6493ad7066e0..6aa08d048da03 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -25,7 +25,7 @@ import com.yammer.metrics.core.Meter import kafka.api._ import kafka.cluster.{BrokerEndPoint, Partition} import kafka.controller.{KafkaController, StateChangeLogger} -import kafka.log._ +import kafka.log.{LeaderHwChange, LogAppendInfo, LogManager, LogReadInfo, UnifiedLog} import kafka.log.remote.RemoteLogManager import kafka.metrics.KafkaMetricsGroup import kafka.server.HostedPartition.Online @@ -61,7 +61,7 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.server.common.MetadataVersion._ -import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel, LogOffsetMetadata, RecordValidationException} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig, LogDirFailureChannel, LogOffsetMetadata, RecordValidationException} import java.nio.file.{Files, Paths} import java.util diff --git a/core/src/main/scala/kafka/server/ZkAdminManager.scala b/core/src/main/scala/kafka/server/ZkAdminManager.scala index 308fa2d1955b9..b669fd556f153 100644 --- a/core/src/main/scala/kafka/server/ZkAdminManager.scala +++ b/core/src/main/scala/kafka/server/ZkAdminManager.scala @@ -18,10 +18,8 @@ package kafka.server import java.util import java.util.Properties - import kafka.admin.{AdminOperationException, AdminUtils} import kafka.common.TopicAlreadyMarkedForDeletionException -import kafka.log.LogConfig import kafka.metrics.KafkaMetricsGroup import kafka.server.ConfigAdminManager.{prepareIncrementalConfigs, toLoggableProps} import kafka.server.DynamicConfig.QuotaConfigs @@ -50,6 +48,7 @@ import org.apache.kafka.common.requests.CreateTopicsRequest._ import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError} import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter} import org.apache.kafka.common.utils.Sanitizer +import org.apache.kafka.server.log.internals.LogConfig import scala.collection.{Map, mutable, _} import scala.jdk.CollectionConverters._ @@ -127,7 +126,7 @@ class ZkAdminManager(val config: KafkaConfig, configs: Properties, assignments: Map[Int, Seq[Int]]): Unit = { metadataAndConfigs.get(topicName).foreach { result => - val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(config), configs) + val logConfig = LogConfig.fromProps(config.extractLogConfigMap, configs) val createEntry = configHelper.createTopicConfigEntry(logConfig, configs, includeSynonyms = false, includeDocumentation = false)(_, _) val topicConfigs = configHelper.allConfigs(logConfig).map { case (k, v) => val entry = createEntry(k, v) @@ -510,7 +509,7 @@ class ZkAdminManager(val config: KafkaConfig, throw new InvalidRequestException("Default topic resources are not allowed.") } val configProps = adminZkClient.fetchEntityConfig(ConfigType.Topic, resource.name) - prepareIncrementalConfigs(alterConfigOps, configProps, LogConfig.configKeys) + prepareIncrementalConfigs(alterConfigOps, configProps, LogConfig.configKeys.asScala) alterTopicConfigs(resource, validateOnly, configProps, configEntriesMap) case ConfigResource.Type.BROKER => diff --git a/core/src/main/scala/kafka/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index 948199521f28c..0f60e3030ec20 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -17,17 +17,16 @@ package kafka.zk import java.util.Properties - import kafka.admin.{AdminOperationException, AdminUtils, BrokerMetadata, RackAwareMode} import kafka.common.TopicAlreadyMarkedForDeletionException import kafka.controller.ReplicaAssignment -import kafka.log.LogConfig import kafka.server.{ConfigEntityName, ConfigType, DynamicConfig} import kafka.utils._ import kafka.utils.Implicits._ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic +import org.apache.kafka.server.log.internals.LogConfig import org.apache.zookeeper.KeeperException.NodeExistsException import scala.collection.{Map, Seq} diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 361c74c5a7c73..e2faccb1482f0 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -21,7 +21,6 @@ import com.yammer.metrics.core.MetricName import kafka.api.LeaderAndIsr import kafka.cluster.Broker import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReplicaAssignment} -import kafka.log.LogConfig import kafka.metrics.KafkaMetricsGroup import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls} import kafka.security.authorizer.AclEntry @@ -36,6 +35,7 @@ import org.apache.kafka.common.security.token.delegation.{DelegationToken, Token import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState +import org.apache.kafka.server.log.internals.LogConfig import org.apache.zookeeper.KeeperException.{Code, NodeExistsException} import org.apache.zookeeper.OpResult.{CheckResult, CreateResult, ErrorResult, SetDataResult} import org.apache.zookeeper.client.ZKClientConfig diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index caf5c4bed20c8..d3da511da68c8 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -16,8 +16,7 @@ package kafka.api import java.util import java.util.Properties import kafka.integration.KafkaServerTestHarness -import kafka.log.LogConfig -import kafka.server.{Defaults, KafkaConfig} +import kafka.server.KafkaConfig import kafka.utils.TestUtils.assertFutureExceptionTypeEquals import kafka.utils.{Logging, TestInfoUtils, TestUtils} import org.apache.kafka.clients.admin.AlterConfigOp.OpType @@ -25,6 +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.log.internals.LogConfig import org.apache.kafka.server.policy.AlterConfigPolicy import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} @@ -86,8 +86,8 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with val topic1 = "describe-alter-configs-topic-1" val topicResource1 = new ConfigResource(ConfigResource.Type.TOPIC, topic1) val topicConfig1 = new Properties - topicConfig1.setProperty(LogConfig.MaxMessageBytesProp, "500000") - topicConfig1.setProperty(LogConfig.RetentionMsProp, "60000000") + topicConfig1.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "500000") + topicConfig1.setProperty(TopicConfig.RETENTION_MS_CONFIG, "60000000") createTopic(topic1, 1, 1, topicConfig1) val topic2 = "describe-alter-configs-topic-2" @@ -132,13 +132,13 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with validations.clear() val topicConfigEntries1 = Seq( - new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.9"), - new ConfigEntry(LogConfig.MinInSyncReplicasProp, "2") // policy doesn't allow this + new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.9"), + new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2") // policy doesn't allow this ).asJava - var topicConfigEntries2 = Seq(new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.8")).asJava + var topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.8")).asJava - val topicConfigEntries3 = Seq(new ConfigEntry(LogConfig.MinInSyncReplicasProp, "-1")).asJava + val topicConfigEntries3 = Seq(new ConfigEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "-1")).asJava val brokerConfigEntries = Seq(new ConfigEntry(KafkaConfig.SslTruststorePasswordProp, "12313")).asJava @@ -165,15 +165,15 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with var configs = describeResult.all.get assertEquals(4, configs.size) - assertEquals(Defaults.LogCleanerMinCleanRatio.toString, configs.get(topicResource1).get(LogConfig.MinCleanableDirtyRatioProp).value) - assertEquals(Defaults.MinInSyncReplicas.toString, configs.get(topicResource1).get(LogConfig.MinInSyncReplicasProp).value) + 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("0.8", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value) + assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) assertNull(configs.get(brokerResource).get(KafkaConfig.SslTruststorePasswordProp).value) // Alter configs with validateOnly = true: only second is valid - topicConfigEntries2 = Seq(new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.7")).asJava + topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.7")).asJava alterResult = client.alterConfigs(Map( topicResource1 -> new Config(topicConfigEntries1), @@ -197,10 +197,10 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with configs = describeResult.all.get assertEquals(4, configs.size) - assertEquals(Defaults.LogCleanerMinCleanRatio.toString, configs.get(topicResource1).get(LogConfig.MinCleanableDirtyRatioProp).value) - assertEquals(Defaults.MinInSyncReplicas.toString, configs.get(topicResource1).get(LogConfig.MinInSyncReplicasProp).value) + 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("0.8", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value) + assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) assertNull(configs.get(brokerResource).get(KafkaConfig.SslTruststorePasswordProp).value) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index ff1b2f5934de0..1b1608dec45ae 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -19,7 +19,6 @@ import java.util.concurrent.ExecutionException import java.util.regex.Pattern import java.util.{Collections, Optional, Properties} import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} -import kafka.log.LogConfig import kafka.security.authorizer.{AclAuthorizer, AclEntry} import kafka.security.authorizer.AclEntry.WildcardHost import kafka.server.{BaseRequestTest, KafkaConfig} @@ -33,7 +32,7 @@ import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter, AclOperation, AclPermissionType} import org.apache.kafka.common.config.internals.BrokerSecurityConfigs -import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig} +import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig} import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic @@ -617,13 +616,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest { new AlterConfigsRequest.Builder( Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic), new AlterConfigsRequest.Config(Collections.singleton( - new AlterConfigsRequest.ConfigEntry(LogConfig.MaxMessageBytesProp, "1000000") + new AlterConfigsRequest.ConfigEntry(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "1000000") ))), true).build() private def incrementalAlterConfigsRequest = { val data = new IncrementalAlterConfigsRequestData val alterableConfig = new AlterableConfig - alterableConfig.setName(LogConfig.MaxMessageBytesProp). + alterableConfig.setName(TopicConfig.MAX_MESSAGE_BYTES_CONFIG). setValue("1000000").setConfigOperation(AlterConfigOp.OpType.SET.id()) val alterableConfigSet = new AlterableConfigCollection alterableConfigSet.add(alterableConfig) diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 65ba9099d265a..a3dae6f666730 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -22,12 +22,12 @@ import java.nio.charset.StandardCharsets import java.util.{Collections, Properties} import java.util.concurrent.TimeUnit import kafka.integration.KafkaServerTestHarness -import kafka.log.LogConfig import kafka.server.KafkaConfig import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.admin.{Admin, NewPartitions} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.network.{ListenerName, Mode} import org.apache.kafka.common.record.TimestampType @@ -254,9 +254,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { // create topic val topicProps = new Properties() if (timestampType == TimestampType.LOG_APPEND_TIME) - topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "LogAppendTime") + topicProps.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime") else - topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "CreateTime") + topicProps.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "CreateTime") TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps) val recordAndFutures = for (i <- 1 to numRecords) yield { diff --git a/core/src/test/scala/integration/kafka/api/ConsumerWithLegacyMessageFormatIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerWithLegacyMessageFormatIntegrationTest.scala index 0ce4004c64e2b..ef0ae9580db7f 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerWithLegacyMessageFormatIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerWithLegacyMessageFormatIntegrationTest.scala @@ -16,10 +16,10 @@ */ package kafka.api -import kafka.log.LogConfig import kafka.server.KafkaConfig import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertThrows} import org.junit.jupiter.api.Test @@ -43,7 +43,7 @@ class ConsumerWithLegacyMessageFormatIntegrationTest extends AbstractConsumerTes val topic2 = "part-test-topic-2" val topic3 = "part-test-topic-3" val props = new Properties() - props.setProperty(LogConfig.MessageFormatVersionProp, "0.9.0") + props.setProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.9.0") createTopic(topic1, numParts, 1) // Topic2 is in old message format. createTopic(topic2, numParts, 1, props) @@ -108,7 +108,7 @@ class ConsumerWithLegacyMessageFormatIntegrationTest extends AbstractConsumerTes val producer = createProducer(configOverrides = prop) createTopicAndSendRecords(producer, topicName = topic0, numPartitions = 2, recordsPerPartition = 100) val props = new Properties() - props.setProperty(LogConfig.MessageFormatVersionProp, "0.9.0") + props.setProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.9.0") createTopic(topic1, numPartitions = 1, replicationFactor = 1, props) sendRecords(producer, numRecords = 100, new TopicPartition(topic1, 0)) diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala index 107442649eea3..d98a1f087ac64 100644 --- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala +++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala @@ -13,15 +13,13 @@ package kafka.api import java.util.{Locale, Properties} - -import kafka.log.LogConfig import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.{JaasTestUtils, TestUtils} import com.yammer.metrics.core.{Gauge, Histogram, Meter} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.{Metric, MetricName, TopicPartition} -import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.config.{SaslConfigs, TopicConfig} import org.apache.kafka.common.errors.{InvalidTopicException, UnknownTopicOrPartitionException} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol @@ -76,7 +74,7 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup { def testMetrics(): Unit = { val topic = "topicWithOldMessageFormat" val props = new Properties - props.setProperty(LogConfig.MessageFormatVersionProp, "0.9.0") + props.setProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.9.0") createTopic(topic, numPartitions = 1, replicationFactor = 1, props) val tp = new TopicPartition(topic, 0) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 2a3e0a1811789..bc2fb9952787e 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -26,7 +26,6 @@ import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} import java.util.{Collections, Optional, Properties} import java.{time, util} import kafka.integration.KafkaServerTestHarness -import kafka.log.LogConfig import kafka.security.authorizer.AclEntry import kafka.server.metadata.KRaftMetadataCache import kafka.server.{Defaults, DynamicConfig, KafkaConfig} @@ -38,13 +37,14 @@ import org.apache.kafka.clients.admin._ import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType} -import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig} +import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig} import org.apache.kafka.common.errors._ import org.apache.kafka.common.requests.{DeleteRecordsRequest, MetadataResponse} import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid} import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT +import org.apache.kafka.server.log.internals.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -377,8 +377,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val topic1 = "describe-alter-configs-topic-1" val topicResource1 = new ConfigResource(ConfigResource.Type.TOPIC, topic1) val topicConfig1 = new Properties - topicConfig1.setProperty(LogConfig.MaxMessageBytesProp, "500000") - topicConfig1.setProperty(LogConfig.RetentionMsProp, "60000000") + topicConfig1.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "500000") + topicConfig1.setProperty(TopicConfig.RETENTION_MS_CONFIG, "60000000") createTopic(topic1, numPartitions = 1, replicationFactor = 1, topicConfig1) val topic2 = "describe-alter-configs-topic-2" @@ -394,19 +394,19 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertEquals(4, configs.size) - val maxMessageBytes1 = configs.get(topicResource1).get(LogConfig.MaxMessageBytesProp) - assertEquals(LogConfig.MaxMessageBytesProp, maxMessageBytes1.name) - assertEquals(topicConfig1.get(LogConfig.MaxMessageBytesProp), maxMessageBytes1.value) + val maxMessageBytes1 = configs.get(topicResource1).get(TopicConfig.MAX_MESSAGE_BYTES_CONFIG) + assertEquals(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageBytes1.name) + assertEquals(topicConfig1.get(TopicConfig.MAX_MESSAGE_BYTES_CONFIG), maxMessageBytes1.value) assertFalse(maxMessageBytes1.isDefault) assertFalse(maxMessageBytes1.isSensitive) assertFalse(maxMessageBytes1.isReadOnly) - assertEquals(topicConfig1.get(LogConfig.RetentionMsProp), - configs.get(topicResource1).get(LogConfig.RetentionMsProp).value) + assertEquals(topicConfig1.get(TopicConfig.RETENTION_MS_CONFIG), + configs.get(topicResource1).get(TopicConfig.RETENTION_MS_CONFIG).value) - val maxMessageBytes2 = configs.get(topicResource2).get(LogConfig.MaxMessageBytesProp) - assertEquals(Defaults.MessageMaxBytes.toString, maxMessageBytes2.value) - assertEquals(LogConfig.MaxMessageBytesProp, maxMessageBytes2.name) + val maxMessageBytes2 = configs.get(topicResource2).get(TopicConfig.MAX_MESSAGE_BYTES_CONFIG) + assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES.toString, maxMessageBytes2.value) + assertEquals(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageBytes2.name) assertTrue(maxMessageBytes2.isDefault) assertFalse(maxMessageBytes2.isSensitive) assertFalse(maxMessageBytes2.isReadOnly) @@ -1810,8 +1810,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val topic1 = "incremental-alter-configs-topic-1" val topic1Resource = new ConfigResource(ConfigResource.Type.TOPIC, topic1) val topic1CreateConfigs = new Properties - topic1CreateConfigs.setProperty(LogConfig.RetentionMsProp, "60000000") - topic1CreateConfigs.setProperty(LogConfig.CleanupPolicyProp, LogConfig.Compact) + topic1CreateConfigs.setProperty(TopicConfig.RETENTION_MS_CONFIG, "60000000") + topic1CreateConfigs.setProperty(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) createTopic(topic1, numPartitions = 1, replicationFactor = 1, topic1CreateConfigs) val topic2 = "incremental-alter-configs-topic-2" @@ -1820,16 +1820,16 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // Alter topic configs var topic1AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.FlushMsProp, "1000"), AlterConfigOp.OpType.SET), - new AlterConfigOp(new ConfigEntry(LogConfig.CleanupPolicyProp, LogConfig.Delete), AlterConfigOp.OpType.APPEND), - new AlterConfigOp(new ConfigEntry(LogConfig.RetentionMsProp, ""), AlterConfigOp.OpType.DELETE) + new AlterConfigOp(new ConfigEntry(TopicConfig.FLUSH_MS_CONFIG, "1000"), AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE), AlterConfigOp.OpType.APPEND), + new AlterConfigOp(new ConfigEntry(TopicConfig.RETENTION_MS_CONFIG, ""), AlterConfigOp.OpType.DELETE) ).asJavaCollection // Test SET and APPEND on previously unset properties var topic2AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.9"), AlterConfigOp.OpType.SET), - new AlterConfigOp(new ConfigEntry(LogConfig.CompressionTypeProp, "lz4"), AlterConfigOp.OpType.SET), - new AlterConfigOp(new ConfigEntry(LogConfig.CleanupPolicyProp, LogConfig.Compact), AlterConfigOp.OpType.APPEND) + new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.9"), AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4"), AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT), AlterConfigOp.OpType.APPEND) ).asJavaCollection var alterResult = client.incrementalAlterConfigs(Map( @@ -1848,23 +1848,23 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertEquals(2, configs.size) - assertEquals("1000", configs.get(topic1Resource).get(LogConfig.FlushMsProp).value) - assertEquals("compact,delete", configs.get(topic1Resource).get(LogConfig.CleanupPolicyProp).value) - assertEquals((Defaults.LogRetentionHours * 60 * 60 * 1000).toString, configs.get(topic1Resource).get(LogConfig.RetentionMsProp).value) + assertEquals("1000", configs.get(topic1Resource).get(TopicConfig.FLUSH_MS_CONFIG).value) + assertEquals("compact,delete", configs.get(topic1Resource).get(TopicConfig.CLEANUP_POLICY_CONFIG).value) + assertEquals(LogConfig.DEFAULT_RETENTION_MS.toString, configs.get(topic1Resource).get(TopicConfig.RETENTION_MS_CONFIG).value) - assertEquals("0.9", configs.get(topic2Resource).get(LogConfig.MinCleanableDirtyRatioProp).value) - assertEquals("lz4", configs.get(topic2Resource).get(LogConfig.CompressionTypeProp).value) - assertEquals("delete,compact", configs.get(topic2Resource).get(LogConfig.CleanupPolicyProp).value) + assertEquals("0.9", configs.get(topic2Resource).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) + assertEquals("lz4", configs.get(topic2Resource).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) + assertEquals("delete,compact", configs.get(topic2Resource).get(TopicConfig.CLEANUP_POLICY_CONFIG).value) // verify subtract operation, including from an empty property topic1AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.CleanupPolicyProp, LogConfig.Compact), AlterConfigOp.OpType.SUBTRACT), - new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, "0"), AlterConfigOp.OpType.SUBTRACT) + new AlterConfigOp(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT), AlterConfigOp.OpType.SUBTRACT), + new AlterConfigOp(new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0"), AlterConfigOp.OpType.SUBTRACT) ).asJava // subtract all from this list property topic2AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.CleanupPolicyProp, LogConfig.Compact + "," + LogConfig.Delete), AlterConfigOp.OpType.SUBTRACT) + new AlterConfigOp(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE), AlterConfigOp.OpType.SUBTRACT) ).asJavaCollection alterResult = client.incrementalAlterConfigs(Map( @@ -1882,14 +1882,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertEquals(2, configs.size) - assertEquals("delete", configs.get(topic1Resource).get(LogConfig.CleanupPolicyProp).value) - assertEquals("1000", configs.get(topic1Resource).get(LogConfig.FlushMsProp).value) // verify previous change is still intact - assertEquals("", configs.get(topic1Resource).get(LogConfig.LeaderReplicationThrottledReplicasProp).value) - assertEquals("", configs.get(topic2Resource).get(LogConfig.CleanupPolicyProp).value ) + assertEquals("delete", configs.get(topic1Resource).get(TopicConfig.CLEANUP_POLICY_CONFIG).value) + assertEquals("1000", configs.get(topic1Resource).get(TopicConfig.FLUSH_MS_CONFIG).value) // verify previous change is still intact + assertEquals("", configs.get(topic1Resource).get(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG).value) + assertEquals("", configs.get(topic2Resource).get(TopicConfig.CLEANUP_POLICY_CONFIG).value ) // Alter topics with validateOnly=true topic1AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.CleanupPolicyProp, LogConfig.Compact), AlterConfigOp.OpType.APPEND) + new AlterConfigOp(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT), AlterConfigOp.OpType.APPEND) ).asJava alterResult = client.incrementalAlterConfigs(Map( @@ -1901,11 +1901,11 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { describeResult = client.describeConfigs(Seq(topic1Resource).asJava) configs = describeResult.all.get - assertEquals("delete", configs.get(topic1Resource).get(LogConfig.CleanupPolicyProp).value) + assertEquals("delete", configs.get(topic1Resource).get(TopicConfig.CLEANUP_POLICY_CONFIG).value) // Alter topics with validateOnly=true with invalid configs topic1AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.CompressionTypeProp, "zip"), AlterConfigOp.OpType.SET) + new AlterConfigOp(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "zip"), AlterConfigOp.OpType.SET) ).asJava alterResult = client.incrementalAlterConfigs(Map( @@ -1935,12 +1935,12 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertNotEquals("", subtractValues) val topicCreateConfigs = new Properties - topicCreateConfigs.setProperty(LogConfig.LeaderReplicationThrottledReplicasProp, appendValues) + topicCreateConfigs.setProperty(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, appendValues) createTopic(topic, numPartitions = 1, replicationFactor = 1, topicCreateConfigs) // Append value that is already present val topicAppendConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, appendValues), AlterConfigOp.OpType.APPEND), + new AlterConfigOp(new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, appendValues), AlterConfigOp.OpType.APPEND), ).asJavaCollection val appendResult = client.incrementalAlterConfigs(Map(topicResource -> topicAppendConfigs).asJava) @@ -1948,7 +1948,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // Subtract values that are not present val topicSubtractConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, subtractValues), AlterConfigOp.OpType.SUBTRACT) + new AlterConfigOp(new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, subtractValues), AlterConfigOp.OpType.SUBTRACT) ).asJavaCollection val subtractResult = client.incrementalAlterConfigs(Map(topicResource -> topicSubtractConfigs).asJava) subtractResult.all.get @@ -1959,7 +1959,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val describeResult = client.describeConfigs(Seq(topicResource).asJava) val configs = describeResult.all.get - assertEquals(appendValues, configs.get(topicResource).get(LogConfig.LeaderReplicationThrottledReplicasProp).value) + assertEquals(appendValues, configs.get(topicResource).get(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG).value) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) @@ -2049,14 +2049,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // Add duplicate Keys for topic1 var topic1AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.75"), AlterConfigOp.OpType.SET), - new AlterConfigOp(new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.65"), AlterConfigOp.OpType.SET), - new AlterConfigOp(new ConfigEntry(LogConfig.CompressionTypeProp, "gzip"), AlterConfigOp.OpType.SET) // valid entry + new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.75"), AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.65"), AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "gzip"), AlterConfigOp.OpType.SET) // valid entry ).asJavaCollection // Add valid config for topic2 var topic2AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.9"), AlterConfigOp.OpType.SET) + new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.9"), AlterConfigOp.OpType.SET) ).asJavaCollection var alterResult = client.incrementalAlterConfigs(Map( @@ -2078,17 +2078,17 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val configs = describeResult.all.get assertEquals(2, configs.size) - assertEquals(Defaults.LogCleanerMinCleanRatio.toString, configs.get(topic1Resource).get(LogConfig.MinCleanableDirtyRatioProp).value) - assertEquals(Defaults.CompressionType, configs.get(topic1Resource).get(LogConfig.CompressionTypeProp).value) - assertEquals("0.9", configs.get(topic2Resource).get(LogConfig.MinCleanableDirtyRatioProp).value) + assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topic1Resource).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(topic1Resource).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) + assertEquals("0.9", configs.get(topic2Resource).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) // Check invalid use of append/subtract operation types topic1AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.CompressionTypeProp, "gzip"), AlterConfigOp.OpType.APPEND) + new AlterConfigOp(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "gzip"), AlterConfigOp.OpType.APPEND) ).asJavaCollection topic2AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.CompressionTypeProp, "snappy"), AlterConfigOp.OpType.SUBTRACT) + new AlterConfigOp(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy"), AlterConfigOp.OpType.SUBTRACT) ).asJavaCollection alterResult = client.incrementalAlterConfigs(Map( @@ -2113,7 +2113,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // Try to add invalid config topic1AlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "1.1"), AlterConfigOp.OpType.SET) + new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "1.1"), AlterConfigOp.OpType.SET) ).asJavaCollection alterResult = client.incrementalAlterConfigs(Map( @@ -2202,15 +2202,15 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { zkClient.getLogConfigs(Set(topic), Collections.emptyMap[String, AnyRef])._1(topic) } - assertEquals(compressionType, logConfig.originals.get(LogConfig.CompressionTypeProp)) - assertNull(logConfig.originals.get(LogConfig.RetentionBytesProp)) - assertEquals(Defaults.LogRetentionBytes, logConfig.retentionSize) + assertEquals(compressionType, logConfig.originals.get(TopicConfig.COMPRESSION_TYPE_CONFIG)) + assertNull(logConfig.originals.get(TopicConfig.RETENTION_BYTES_CONFIG)) + assertEquals(LogConfig.DEFAULT_RETENTION_BYTES, logConfig.retentionSize) } client = Admin.create(createConfig) val invalidConfigs = Map[String, String]( - LogConfig.RetentionBytesProp -> null, - LogConfig.CompressionTypeProp -> "producer" + TopicConfig.RETENTION_BYTES_CONFIG -> null, + TopicConfig.COMPRESSION_TYPE_CONFIG -> "producer" ).asJava val newTopic = new NewTopic(topic, 2, brokerCount.toShort) assertFutureExceptionTypeEquals( @@ -2219,15 +2219,15 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { Some("Null value not supported for topic configs: retention.bytes") ) - val validConfigs = Map[String, String](LogConfig.CompressionTypeProp -> "producer").asJava + val validConfigs = Map[String, String](TopicConfig.COMPRESSION_TYPE_CONFIG -> "producer").asJava client.createTopics(Collections.singletonList(newTopic.configs(validConfigs))).all.get() waitForTopics(client, expectedPresent = Seq(topic), expectedMissing = List()) validateLogConfig(compressionType = "producer") val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic) val alterOps = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.RetentionBytesProp, null), AlterConfigOp.OpType.SET), - new AlterConfigOp(new ConfigEntry(LogConfig.CompressionTypeProp, "lz4"), AlterConfigOp.OpType.SET) + new AlterConfigOp(new ConfigEntry(TopicConfig.RETENTION_BYTES_CONFIG, null), AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4"), AlterConfigOp.OpType.SET) ) assertFutureExceptionTypeEquals( client.incrementalAlterConfigs(Map(topicResource -> alterOps.asJavaCollection).asJava).all, @@ -2476,7 +2476,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testAppendConfigToExistentValue(ignored: String): Unit = { val props = new Properties(); - props.setProperty(LogConfig.LeaderReplicationThrottledReplicasProp, "1:1") + props.setProperty(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "1:1") testAppendConfig(props, "0:0", "1:1,0:0") } @@ -2485,7 +2485,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { createTopic(topic, topicConfig = props) val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic) val topicAlterConfigs = Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, append), AlterConfigOp.OpType.APPEND), + new AlterConfigOp(new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, append), AlterConfigOp.OpType.APPEND), ).asJavaCollection val alterResult = client.incrementalAlterConfigs(Map( @@ -2494,7 +2494,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { alterResult.all().get(15, TimeUnit.SECONDS) ensureConsistentKRaftMetadata() - val config = client.describeConfigs(List(topicResource).asJava).all().get().get(topicResource).get(LogConfig.LeaderReplicationThrottledReplicasProp) + val config = client.describeConfigs(List(topicResource).asJava).all().get().get(topicResource).get(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG) assertEquals(expected, config.value()) } @@ -2543,7 +2543,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val newTopics = Seq(new NewTopic("foo", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava). - configs(Collections.singletonMap(LogConfig.IndexIntervalBytesProp, "9999999")), + configs(Collections.singletonMap(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "9999999")), new NewTopic("bar", 3, 3.toShort), new NewTopic("baz", Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava) ) @@ -2561,24 +2561,24 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val topicConfigs = result.config("foo").get() // From the topic configuration defaults. - assertEquals(new ConfigEntry(LogConfig.CleanupPolicyProp, "delete", + assertEquals(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, "delete", ConfigSource.DEFAULT_CONFIG, false, false, Collections.emptyList(), null, null), - topicConfigs.get(LogConfig.CleanupPolicyProp)) + topicConfigs.get(TopicConfig.CLEANUP_POLICY_CONFIG)) // From dynamic cluster config via the synonym LogRetentionTimeHoursProp. - assertEquals(new ConfigEntry(LogConfig.RetentionMsProp, "10800000", + assertEquals(new ConfigEntry(TopicConfig.RETENTION_MS_CONFIG, "10800000", ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG, false, false, Collections.emptyList(), null, null), - topicConfigs.get(LogConfig.RetentionMsProp)) + topicConfigs.get(TopicConfig.RETENTION_MS_CONFIG)) // From dynamic broker config via LogCleanerDeleteRetentionMsProp. - assertEquals(new ConfigEntry(LogConfig.DeleteRetentionMsProp, "34", + assertEquals(new ConfigEntry(TopicConfig.DELETE_RETENTION_MS_CONFIG, "34", ConfigSource.DYNAMIC_BROKER_CONFIG, false, false, Collections.emptyList(), null, null), - topicConfigs.get(LogConfig.DeleteRetentionMsProp)) + topicConfigs.get(TopicConfig.DELETE_RETENTION_MS_CONFIG)) // From static broker config by SegmentJitterMsProp. - assertEquals(new ConfigEntry(LogConfig.SegmentJitterMsProp, "123", + assertEquals(new ConfigEntry(TopicConfig.SEGMENT_JITTER_MS_CONFIG, "123", ConfigSource.STATIC_BROKER_CONFIG, false, false, Collections.emptyList(), null, null), - topicConfigs.get(LogConfig.SegmentJitterMsProp)) + topicConfigs.get(TopicConfig.SEGMENT_JITTER_MS_CONFIG)) // From static broker config by the synonym LogRollTimeHoursProp. val segmentMsPropType = if (isKRaftTest()) { @@ -2586,14 +2586,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } else { ConfigSource.DEFAULT_CONFIG } - assertEquals(new ConfigEntry(LogConfig.SegmentMsProp, "7200000", + assertEquals(new ConfigEntry(TopicConfig.SEGMENT_MS_CONFIG, "7200000", segmentMsPropType, false, false, Collections.emptyList(), null, null), - topicConfigs.get(LogConfig.SegmentMsProp)) + topicConfigs.get(TopicConfig.SEGMENT_MS_CONFIG)) // From the dynamic topic config. - assertEquals(new ConfigEntry(LogConfig.IndexIntervalBytesProp, "9999999", + assertEquals(new ConfigEntry(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "9999999", ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, Collections.emptyList(), null, null), - topicConfigs.get(LogConfig.IndexIntervalBytesProp)) + topicConfigs.get(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG)) } } @@ -2608,12 +2608,12 @@ object PlaintextAdminIntegrationTest { ): Unit = { // Alter topics var topicConfigEntries1 = Seq( - new ConfigEntry(LogConfig.FlushMsProp, "1000") + new ConfigEntry(TopicConfig.FLUSH_MS_CONFIG, "1000") ).asJava var topicConfigEntries2 = Seq( - new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.9"), - new ConfigEntry(LogConfig.CompressionTypeProp, "lz4") + new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.9"), + new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4") ).asJava var alterResult = admin.alterConfigs(Map( @@ -2631,22 +2631,21 @@ object PlaintextAdminIntegrationTest { assertEquals(2, configs.size) - assertEquals("1000", configs.get(topicResource1).get(LogConfig.FlushMsProp).value) - assertEquals(Defaults.MessageMaxBytes.toString, - configs.get(topicResource1).get(LogConfig.MaxMessageBytesProp).value) - assertEquals((Defaults.LogRetentionHours * 60 * 60 * 1000).toString, - configs.get(topicResource1).get(LogConfig.RetentionMsProp).value) + assertEquals("1000", configs.get(topicResource1).get(TopicConfig.FLUSH_MS_CONFIG).value) + assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES.toString, + configs.get(topicResource1).get(TopicConfig.MAX_MESSAGE_BYTES_CONFIG).value) + assertEquals(LogConfig.DEFAULT_RETENTION_MS.toString, configs.get(topicResource1).get(TopicConfig.RETENTION_MS_CONFIG).value) - assertEquals("0.9", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value) - assertEquals("lz4", configs.get(topicResource2).get(LogConfig.CompressionTypeProp).value) + assertEquals("0.9", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) + assertEquals("lz4", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) // Alter topics with validateOnly=true topicConfigEntries1 = Seq( - new ConfigEntry(LogConfig.MaxMessageBytesProp, "10") + new ConfigEntry(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "10") ).asJava topicConfigEntries2 = Seq( - new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "0.3") + new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0.3") ).asJava alterResult = admin.alterConfigs(Map( @@ -2664,9 +2663,9 @@ object PlaintextAdminIntegrationTest { assertEquals(2, configs.size) - assertEquals(Defaults.MessageMaxBytes.toString, - configs.get(topicResource1).get(LogConfig.MaxMessageBytesProp).value) - assertEquals("0.9", configs.get(topicResource2).get(LogConfig.MinCleanableDirtyRatioProp).value) + assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES.toString, + configs.get(topicResource1).get(TopicConfig.MAX_MESSAGE_BYTES_CONFIG).value) + assertEquals("0.9", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) } @nowarn("cat=deprecation") @@ -2684,11 +2683,11 @@ object PlaintextAdminIntegrationTest { createTopicWithAdmin(admin, topic2, test.brokers, numPartitions = 1, replicationFactor = 1) val topicConfigEntries1 = Seq( - new ConfigEntry(LogConfig.MinCleanableDirtyRatioProp, "1.1"), // this value is invalid as it's above 1.0 - new ConfigEntry(LogConfig.CompressionTypeProp, "lz4") + new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "1.1"), // this value is invalid as it's above 1.0 + new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4") ).asJava - var topicConfigEntries2 = Seq(new ConfigEntry(LogConfig.CompressionTypeProp, "snappy")).asJava + var topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy")).asJava val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, test.brokers.head.config.brokerId.toString) val brokerConfigEntries = Seq(new ConfigEntry(KafkaConfig.ZkConnectProp, "localhost:2181")).asJava @@ -2711,17 +2710,17 @@ object PlaintextAdminIntegrationTest { var configs = describeResult.all.get assertEquals(3, configs.size) - assertEquals(Defaults.LogCleanerMinCleanRatio.toString, - configs.get(topicResource1).get(LogConfig.MinCleanableDirtyRatioProp).value) - assertEquals(Defaults.CompressionType, - configs.get(topicResource1).get(LogConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, + configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, + configs.get(topicResource1).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals("snappy", configs.get(topicResource2).get(LogConfig.CompressionTypeProp).value) + assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(Defaults.CompressionType, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) // Alter configs with validateOnly = true: first and third are invalid, second is valid - topicConfigEntries2 = Seq(new ConfigEntry(LogConfig.CompressionTypeProp, "gzip")).asJava + topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "gzip")).asJava alterResult = admin.alterConfigs(Map( topicResource1 -> new Config(topicConfigEntries1), @@ -2740,13 +2739,13 @@ object PlaintextAdminIntegrationTest { configs = describeResult.all.get assertEquals(3, configs.size) - assertEquals(Defaults.LogCleanerMinCleanRatio.toString, - configs.get(topicResource1).get(LogConfig.MinCleanableDirtyRatioProp).value) - assertEquals(Defaults.CompressionType, - configs.get(topicResource1).get(LogConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, + configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, + configs.get(topicResource1).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals("snappy", configs.get(topicResource2).get(LogConfig.CompressionTypeProp).value) + assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(Defaults.CompressionType, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) } } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 740fd8a0780e3..017b950435d19 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -17,7 +17,6 @@ import java.util import java.util.Arrays.asList import java.util.regex.Pattern import java.util.{Locale, Optional, Properties} -import kafka.log.LogConfig import kafka.utils.TestUtils import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} @@ -37,6 +36,7 @@ import kafka.server.QuotaType import kafka.server.KafkaServer import org.apache.kafka.clients.admin.NewPartitions import org.apache.kafka.clients.admin.NewTopic +import org.apache.kafka.common.config.TopicConfig import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -1305,7 +1305,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { def testConsumeMessagesWithLogAppendTime(): Unit = { val topicName = "testConsumeMessagesWithLogAppendTime" val topicProps = new Properties() - topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "LogAppendTime") + topicProps.setProperty(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "LogAppendTime") createTopic(topicName, 2, 2, topicProps) val startTime = System.currentTimeMillis() diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index 7c89db8fee4f5..c6caf48c3667f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -19,13 +19,13 @@ package kafka.api import java.util.Properties import java.util.concurrent.{ExecutionException, Future, TimeUnit} -import kafka.log.LogConfig -import kafka.server.Defaults import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.producer.{BufferExhaustedException, KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata} +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.{InvalidTimestampException, RecordTooLargeException, SerializationException, TimeoutException} import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch, Records, TimestampType} import org.apache.kafka.common.serialization.ByteArraySerializer +import org.apache.kafka.server.log.internals.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Timeout import org.junit.jupiter.params.ParameterizedTest @@ -124,7 +124,7 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { @ValueSource(strings = Array("zk", "kraft")) def testSendWithInvalidCreateTime(quorum: String): Unit = { val topicProps = new Properties() - topicProps.setProperty(LogConfig.MessageTimestampDifferenceMaxMsProp, "1000") + topicProps.setProperty(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000") TestUtils.createTopicWithAdmin(admin, topic, brokers, 1, 2, topicConfig = topicProps) val producer = createProducer() @@ -218,7 +218,7 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { val valueLengthSize = 3 val overhead = Records.LOG_OVERHEAD + DefaultRecordBatch.RECORD_BATCH_OVERHEAD + DefaultRecord.MAX_RECORD_OVERHEAD + keyLengthSize + headerLengthSize + valueLengthSize - val valueSize = Defaults.MessageMaxBytes - overhead + val valueSize = LogConfig.DEFAULT_MAX_MESSAGE_BYTES - overhead val record0 = new ProducerRecord(topic, new Array[Byte](0), new Array[Byte](valueSize)) assertEquals(record0.value.length, producer.send(record0).get.serializedValueSize) diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index e04dd0802a374..f5c0ef8c93b2e 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -19,12 +19,11 @@ package kafka.api import java.util.concurrent.ExecutionException import java.util.Properties - import kafka.integration.KafkaServerTestHarness -import kafka.log.LogConfig import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch} @@ -114,8 +113,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { private def checkTooLargeRecordForReplicationWithAckAll(maxFetchSize: Int): Unit = { val maxMessageSize = maxFetchSize + 100 val topicConfig = new Properties - topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, numServers.toString) - topicConfig.setProperty(LogConfig.MaxMessageBytesProp, maxMessageSize.toString) + topicConfig.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, numServers.toString) + topicConfig.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageSize.toString) // create topic val topic10 = "topic10" diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 2b09ce4b7303b..e4b61f3afc570 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -13,11 +13,9 @@ package kafka.api import java.util - -import kafka.log.LogConfig import kafka.security.authorizer.AclAuthorizer import kafka.security.authorizer.AclEntry.{WildcardHost, WildcardPrincipalString} -import kafka.server.{Defaults, KafkaConfig} +import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import kafka.utils.TestUtils._ import org.apache.kafka.clients.admin._ @@ -25,17 +23,18 @@ import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl._ import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE} import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} -import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException, TopicAuthorizationException, UnknownTopicOrPartitionException} import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC} import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.log.internals.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} -import java.util.Collections +import java.util.Collections import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.compat.java8.OptionConverters._ @@ -391,7 +390,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get() val topics = Seq(topic1, topic2) - val configsOverride = Map(LogConfig.SegmentBytesProp -> "100000").asJava + val configsOverride = Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "100000").asJava val newTopics = Seq( new NewTopic(topic1, 2, 3.toShort).configs(configsOverride), new NewTopic(topic2, Option.empty[Integer].asJava, Option.empty[java.lang.Short].asJava).configs(configsOverride)) @@ -404,11 +403,11 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu assertEquals(3, result.replicationFactor(topic1).get()) val topicConfigs = result.config(topic1).get().entries.asScala assertTrue(topicConfigs.nonEmpty) - val segmentBytesConfig = topicConfigs.find(_.name == LogConfig.SegmentBytesProp).get + val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_CONFIG).get assertEquals(100000, segmentBytesConfig.value.toLong) assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) - val compressionConfig = topicConfigs.find(_.name == LogConfig.CompressionTypeProp).get - assertEquals(Defaults.CompressionType, compressionConfig.value) + val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, compressionConfig.value) assertEquals(ConfigEntry.ConfigSource.DEFAULT_CONFIG, compressionConfig.source) assertFutureExceptionTypeEquals(result.numPartitions(topic2), classOf[TopicAuthorizationException]) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 88654090d2b79..3f1f1c8054d7d 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -31,7 +31,7 @@ import com.yammer.metrics.core.MetricName import kafka.admin.ConfigCommand import kafka.api.{KafkaSasl, SaslSetup} import kafka.controller.{ControllerBrokerStateInfo, ControllerChannelManager} -import kafka.log.{CleanerConfig, LogConfig, UnifiedLog} +import kafka.log.UnifiedLog import kafka.network.{Processor, RequestChannel} import kafka.utils._ import kafka.utils.Implicits._ @@ -59,6 +59,7 @@ import org.apache.kafka.common.requests.MetadataRequest import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.scram.ScramCredential import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} +import org.apache.kafka.server.log.internals.{CleanerConfig, LogConfig} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} @@ -552,7 +553,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertEquals(300000, newCleanerConfig.ioBufferSize) assertEquals(40000, newCleanerConfig.maxMessageSize) assertEquals(50000000, newCleanerConfig.maxIoBytesPerSecond, 50000000) - assertEquals(6000, newCleanerConfig.backOffMs) + assertEquals(6000, newCleanerConfig.backoffMs) // Verify thread count verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 2) @@ -651,7 +652,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } // Verify that configs of existing logs have been updated - val newLogConfig = LogConfig(LogConfig.extractLogConfigMap(servers.head.config)) + val newLogConfig = new LogConfig(servers.head.config.extractLogConfigMap) TestUtils.waitUntilTrue(() => servers.head.logManager.currentDefaultConfig == newLogConfig, "Config not updated in LogManager") @@ -702,7 +703,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup assertEquals(500000, servers.head.config.values.get(KafkaConfig.LogIndexSizeMaxBytesProp)) assertEquals(TimeUnit.DAYS.toMillis(2), servers.head.config.values.get(KafkaConfig.LogRetentionTimeMillisProp)) servers.tail.foreach { server => - assertEquals(Defaults.LogIndexSizeMaxBytes, server.config.values.get(KafkaConfig.LogIndexSizeMaxBytesProp)) + assertEquals(LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, server.config.values.get(KafkaConfig.LogIndexSizeMaxBytesProp)) assertEquals(1680000000L, server.config.values.get(KafkaConfig.LogRetentionTimeMillisProp)) } @@ -722,7 +723,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup 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 == Defaults.LogIndexSizeMaxBytes && log.config.retentionMs == 1680000000L, + TestUtils.waitUntilTrue(() => log.config.maxIndexSize == LogConfig.DEFAULT_SEGMENT_INDEX_BYTES && log.config.retentionMs == 1680000000L, "Existing topic config using defaults not updated") } } diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 7cc63f0e0a048..7b7a173373cdc 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -16,7 +16,7 @@ */ package kafka.raft -import kafka.log.{Defaults, SegmentDeletion, UnifiedLog} +import kafka.log.{SegmentDeletion, UnifiedLog} import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMillisProp, MetadataLogSegmentMinBytesProp, NodeIdProp, ProcessRolesProp, QuorumVotersProp} import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.{MockTime, TestUtils} @@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.raft.internals.BatchBuilder import org.apache.kafka.raft._ import org.apache.kafka.server.common.serialization.RecordSerde +import org.apache.kafka.server.log.internals.LogConfig import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import org.apache.kafka.test.TestUtils.assertOptional import org.junit.jupiter.api.Assertions._ @@ -839,7 +840,7 @@ final class KafkaMetadataLogTest { retentionMillis = 60 * 1000, maxBatchSizeInBytes = 512, maxFetchSizeInBytes = DefaultMetadataLogConfig.maxFetchSizeInBytes, - fileDeleteDelayMs = Defaults.FileDeleteDelayMs, + fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, nodeId = 1 ) config.copy() @@ -988,7 +989,7 @@ final class KafkaMetadataLogTest { } // Sleep long enough to trigger a possible segment delete because of the default retention - val defaultLogRetentionMs = Defaults.RetentionMs * 2 + val defaultLogRetentionMs = LogConfig.DEFAULT_RETENTION_MS * 2 mockTime.sleep(defaultLogRetentionMs) assertTrue(log.maybeClean()) @@ -1021,7 +1022,7 @@ object KafkaMetadataLogTest { retentionMillis = 60 * 1000, maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES, maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES, - fileDeleteDelayMs = Defaults.FileDeleteDelayMs, + fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, nodeId = 1 ) diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index 5492a15febabf..792b3550fd9c3 100755 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -19,14 +19,14 @@ package kafka import java.util.Properties import java.util.concurrent.atomic._ - import kafka.log._ import kafka.server.{BrokerTopicStats, FetchLogEnd} import kafka.utils._ import org.apache.kafka.clients.consumer.OffsetOutOfRangeException +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.FileRecords import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{LogConfig, LogDirFailureChannel} /** * A stress test that instantiates a log and then runs continual appends against it from one thread and continual reads against it @@ -39,12 +39,12 @@ object StressTestLog { val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir()) val time = new MockTime val logProperties = new Properties() - logProperties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer) - logProperties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer) - logProperties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer) + logProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, 64*1024*1024: java.lang.Integer) + logProperties.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, Int.MaxValue: java.lang.Integer) + logProperties.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1024*1024: java.lang.Integer) val log = UnifiedLog(dir = dir, - config = LogConfig(logProperties), + config = new LogConfig(logProperties), logStartOffset = 0L, recoveryPoint = 0L, scheduler = time.scheduler, diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index ab681e690afd6..672dea642f26b 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -22,14 +22,14 @@ import java.nio._ import java.nio.channels._ import java.nio.file.StandardOpenOption import java.util.{Properties, Random} - import joptsimple._ import kafka.log._ import kafka.server.BrokerTopicStats import kafka.utils._ +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{LogConfig, LogDirFailureChannel} import scala.math._ @@ -121,8 +121,8 @@ object TestLinearWriteSpeed { } else if(options.has(logOpt)) { val segmentSize = rand.nextInt(512)*1024*1024 + 64*1024*1024 // vary size to avoid herd effect val logProperties = new Properties() - logProperties.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) - logProperties.put(LogConfig.FlushMessagesProp, flushInterval: java.lang.Long) + logProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentSize: java.lang.Integer) + logProperties.put(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, flushInterval: java.lang.Long) writables(i) = new LogWritable(new File(dir, "kafka-test-" + i), new LogConfig(logProperties), scheduler, messageSet) } else { System.err.println("Must specify what to write to with one of --log, --channel, or --mmap") diff --git a/core/src/test/scala/unit/kafka/admin/ReplicationQuotaUtils.scala b/core/src/test/scala/unit/kafka/admin/ReplicationQuotaUtils.scala index 4ac64fe1a58a8..a0f28f9071882 100644 --- a/core/src/test/scala/unit/kafka/admin/ReplicationQuotaUtils.scala +++ b/core/src/test/scala/unit/kafka/admin/ReplicationQuotaUtils.scala @@ -12,10 +12,10 @@ */ package kafka.admin -import kafka.log.LogConfig import kafka.server.{ConfigType, DynamicConfig, KafkaServer} import kafka.utils.TestUtils import kafka.zk.AdminZkClient +import org.apache.kafka.server.log.internals.LogConfig import scala.collection.Seq @@ -29,8 +29,8 @@ object ReplicationQuotaUtils { brokerConfig.contains(DynamicConfig.Broker.FollowerReplicationThrottledRateProp) } val topicConfig = adminZkClient.fetchEntityConfig(ConfigType.Topic, topic) - val hasReplicasProp = topicConfig.contains(LogConfig.LeaderReplicationThrottledReplicasProp) || - topicConfig.contains(LogConfig.FollowerReplicationThrottledReplicasProp) + val hasReplicasProp = topicConfig.contains(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG) || + topicConfig.contains(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG) !hasRateProp && !hasReplicasProp }, "Throttle limit/replicas was not unset") } @@ -47,8 +47,8 @@ object ReplicationQuotaUtils { } //Check replicas assigned val topicConfig = adminZkClient.fetchEntityConfig(ConfigType.Topic, topic) - val leader = topicConfig.getProperty(LogConfig.LeaderReplicationThrottledReplicasProp).split(",").toSet - val follower = topicConfig.getProperty(LogConfig.FollowerReplicationThrottledReplicasProp).split(",").toSet + val leader = topicConfig.getProperty(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG).split(",").toSet + val follower = topicConfig.getProperty(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG).split(",").toSet val topicConfigAvailable = leader == throttledLeaders && follower == throttledFollowers brokerConfigAvailable && topicConfigAvailable }, "throttle limit/replicas was not set") diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index 13e627b529851..d72dafadb3a13 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -16,23 +16,25 @@ */ package kafka.cluster -import kafka.log.{CleanerConfig, LogConfig, LogManager} +import kafka.log.LogManager import kafka.server.{Defaults, MetadataCache} import kafka.server.checkpoints.OffsetCheckpoints import kafka.server.metadata.MockConfigRepository -import kafka.utils.TestUtils.{MockAlterPartitionManager, MockAlterPartitionListener} +import kafka.utils.TestUtils.{MockAlterPartitionListener, MockAlterPartitionManager} import kafka.utils.{MockTime, TestUtils} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.utils.Utils import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.mockito.ArgumentMatchers import org.mockito.Mockito.{mock, when} + import java.io.File import java.util.Properties - import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.{CleanerConfig, LogConfig} import scala.jdk.CollectionConverters._ @@ -64,14 +66,14 @@ class AbstractPartitionTest { TestUtils.clearYammerMetrics() val logProps = createLogProperties(Map.empty) - logConfig = LogConfig(logProps) + logConfig = new LogConfig(logProps) configRepository = MockConfigRepository.forTopic(topicPartition.topic, logProps) tmpDir = TestUtils.tempDir() logDir1 = TestUtils.randomPartitionLogDir(tmpDir) logDir2 = TestUtils.randomPartitionLogDir(tmpDir) logManager = TestUtils.createLogManager(Seq(logDir1, logDir2), logConfig, configRepository, - CleanerConfig(enableCleaner = false), time, interBrokerProtocolVersion) + new CleanerConfig(false), time, interBrokerProtocolVersion) logManager.startup(Set.empty) alterPartitionManager = TestUtils.createAlterIsrManager() @@ -95,9 +97,9 @@ class AbstractPartitionTest { def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 512: java.lang.Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) - logProps.put(LogConfig.RetentionMsProp, 999: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 999: java.lang.Integer) overrides.foreach { case (k, v) => logProps.put(k, v) } logProps } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 22f2e75cb5817..667d083c448b0 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -28,6 +28,7 @@ import kafka.server.epoch.LeaderEpochFileCache import kafka.server.metadata.MockConfigRepository import kafka.utils._ import org.apache.kafka.common.TopicIdPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} @@ -35,7 +36,7 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} +import org.apache.kafka.server.log.internals.{AppendOrigin, CleanerConfig, LogConfig, LogDirFailureChannel} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers @@ -75,7 +76,7 @@ class PartitionLockTest extends Logging { val logConfig = new LogConfig(new Properties) val configRepository = MockConfigRepository.forTopic(topicPartition.topic, createLogProperties(Map.empty)) logManager = TestUtils.createLogManager(Seq(logDir), logConfig, configRepository, - CleanerConfig(enableCleaner = false), mockTime) + new CleanerConfig(false), mockTime) partition = setupPartitionWithMocks(logManager) } @@ -355,9 +356,9 @@ class PartitionLockTest extends Logging { private def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 512: java.lang.Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) - logProps.put(LogConfig.RetentionMsProp, 999: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 999: java.lang.Integer) overrides.foreach { case (k, v) => logProps.put(k, v) } logProps } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 2c52888bdbed7..cd7c5cc4b921c 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -19,7 +19,7 @@ package kafka.cluster import java.net.InetAddress import com.yammer.metrics.core.Metric import kafka.common.UnexpectedAppendOffsetException -import kafka.log.{Defaults => _, _} +import kafka.log._ import kafka.server._ import kafka.server.checkpoints.OffsetCheckpoints import kafka.server.epoch.EpochEntry @@ -54,7 +54,7 @@ import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 -import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} +import org.apache.kafka.server.log.internals.{AppendOrigin, CleanerConfig, LogDirFailureChannel} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -2458,7 +2458,7 @@ class PartitionTest extends AbstractPartitionTest { val spyConfigRepository = spy(configRepository) logManager = TestUtils.createLogManager( logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, configRepository = spyConfigRepository, - cleanerConfig = CleanerConfig(enableCleaner = false), time = time) + cleanerConfig = new CleanerConfig(false), time = time) val spyLogManager = spy(logManager) val partition = new Partition(topicPartition, replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, @@ -2491,7 +2491,7 @@ class PartitionTest extends AbstractPartitionTest { val spyConfigRepository = spy(configRepository) logManager = TestUtils.createLogManager( logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, configRepository = spyConfigRepository, - cleanerConfig = CleanerConfig(enableCleaner = false), time = time) + cleanerConfig = new CleanerConfig(false), time = time) val spyLogManager = spy(logManager) doAnswer((_: InvocationOnMock) => { logManager.initializingLog(topicPartition) @@ -2530,7 +2530,7 @@ class PartitionTest extends AbstractPartitionTest { val spyConfigRepository = spy(configRepository) logManager = TestUtils.createLogManager( logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, configRepository = spyConfigRepository, - cleanerConfig = CleanerConfig(enableCleaner = false), time = time) + cleanerConfig = new CleanerConfig(false), time = time) logManager.startup(Set.empty) val spyLogManager = spy(logManager) diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionWithLegacyMessageFormatTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionWithLegacyMessageFormatTest.scala index 75fec767abd5b..8185781c135ce 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionWithLegacyMessageFormatTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionWithLegacyMessageFormatTest.scala @@ -16,14 +16,14 @@ */ package kafka.cluster -import kafka.log.LogConfig import kafka.utils.TestUtils +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{RecordVersion, SimpleRecord} import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Test -import java.util.Optional +import java.util.Optional import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_8_IV1 @@ -39,7 +39,7 @@ class PartitionWithLegacyMessageFormatTest extends AbstractPartitionTest { def testMakeLeaderDoesNotUpdateEpochCacheForOldFormats(): Unit = { val leaderEpoch = 8 configRepository.setTopicConfig(topicPartition.topic(), - LogConfig.MessageFormatVersionProp, MetadataVersion.IBP_0_10_2_IV0.shortVersion) + TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, MetadataVersion.IBP_0_10_2_IV0.shortVersion) val log = logManager.getOrCreateLog(topicPartition, topicId = None) log.appendAsLeader(TestUtils.records(List( new SimpleRecord("k1".getBytes, "v1".getBytes), diff --git a/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala b/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala index 566d3173a33be..d94511b65b4bd 100644 --- a/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala +++ b/core/src/test/scala/unit/kafka/controller/PartitionStateMachineTest.scala @@ -17,7 +17,6 @@ package kafka.controller import kafka.api.LeaderAndIsr -import kafka.log.LogConfig import kafka.server.KafkaConfig import kafka.utils.TestUtils import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult @@ -25,6 +24,7 @@ import kafka.zk.{KafkaZkClient, TopicPartitionStateZNode} import kafka.zookeeper._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.server.common.MetadataVersion.{IBP_3_1_IV0, IBP_3_2_IV0} +import org.apache.kafka.server.log.internals.LogConfig import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.data.Stat import org.junit.jupiter.api.Assertions._ @@ -34,6 +34,8 @@ import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito.{mock, verify, when} +import java.util.Properties + class PartitionStateMachineTest { private var controllerContext: ControllerContext = _ private var mockZkClient: KafkaZkClient = _ @@ -257,7 +259,7 @@ class PartitionStateMachineTest { TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch), stat, ResponseMetadata(0, 0)))) when(mockZkClient.getLogConfigs(Set.empty, config.originals())) - .thenReturn((Map(partition.topic -> LogConfig()), Map.empty[String, Exception])) + .thenReturn((Map(partition.topic -> new LogConfig(new Properties)), Map.empty[String, Exception])) val leaderAndIsrAfterElection = leaderAndIsr.newLeader(brokerId) val updatedLeaderAndIsr = leaderAndIsrAfterElection.withPartitionEpoch(2) when(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion)) diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index 326ca5101ce0d..0d4fd4a0e61bc 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -22,7 +22,7 @@ import java.util.{Collections, Random} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.Lock import kafka.coordinator.AbstractCoordinatorConcurrencyTest._ -import kafka.log.{LogConfig, UnifiedLog} +import kafka.log.UnifiedLog import kafka.server._ import kafka.utils._ import kafka.utils.timer.MockTimer @@ -31,9 +31,9 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordConversionStats} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse -import org.apache.kafka.server.log.internals.AppendOrigin +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig} import org.junit.jupiter.api.{AfterEach, BeforeEach} -import org.mockito.Mockito.{mock, withSettings, CALLS_REAL_METHODS} +import org.mockito.Mockito.{CALLS_REAL_METHODS, mock, withSettings} import scala.collection._ import scala.jdk.CollectionConverters._ diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index ffd9f506fbacc..24798bd3df75a 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -19,11 +19,10 @@ package kafka.coordinator.transaction import java.nio.ByteBuffer import java.util.Collections import java.util.concurrent.atomic.AtomicBoolean - import kafka.coordinator.AbstractCoordinatorConcurrencyTest import kafka.coordinator.AbstractCoordinatorConcurrencyTest._ import kafka.coordinator.transaction.TransactionCoordinatorConcurrencyTest._ -import kafka.log.{LogConfig, UnifiedLog} +import kafka.log.UnifiedLog import kafka.server.{FetchDataInfo, FetchLogEnd, KafkaConfig, MetadataCache, RequestLocal} import kafka.utils.{Pool, TestUtils} import org.apache.kafka.clients.{ClientResponse, NetworkClient} @@ -35,7 +34,7 @@ import org.apache.kafka.common.record.{CompressionType, FileRecords, MemoryRecor import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} import org.apache.kafka.common.{Node, TopicPartition} -import org.apache.kafka.server.log.internals.LogOffsetMetadata +import org.apache.kafka.server.log.internals.{LogConfig, LogOffsetMetadata} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.{ArgumentCaptor, ArgumentMatchers} diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 397c6c2c75b3b..9e6b24ddb5f1d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.util.concurrent.CountDownLatch import java.util.concurrent.locks.ReentrantLock import javax.management.ObjectName -import kafka.log.{Defaults, LogConfig, UnifiedLog} +import kafka.log.UnifiedLog import kafka.server.{FetchDataInfo, FetchLogEnd, ReplicaManager, RequestLocal} import kafka.utils.{MockScheduler, Pool, TestUtils} import kafka.zk.KafkaZkClient @@ -33,7 +33,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.MockTime -import org.apache.kafka.server.log.internals.{AppendOrigin, LogOffsetMetadata} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig, LogOffsetMetadata} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.{ArgumentCaptor, ArgumentMatchers} @@ -932,7 +932,7 @@ class TransactionStateManagerTest { val partitionIds = 0 until numPartitions loadTransactionsForPartitions(partitionIds) - expectLogConfig(partitionIds, Defaults.MaxMessageSize) + expectLogConfig(partitionIds, LogConfig.DEFAULT_MAX_MESSAGE_BYTES) txnMetadata1.txnLastUpdateTimestamp = time.milliseconds() - txnConfig.transactionalIdExpirationMs txnMetadata1.state = txnState diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index d6f97f36e00ef..8290c0fa0dbd4 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -23,9 +23,10 @@ import kafka.server.BrokerTopicStats import kafka.utils.{MockTime, Pool, TestUtils} import kafka.utils.Implicits._ import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{CleanerConfig, LogConfig, LogDirFailureChannel} import org.junit.jupiter.api.{AfterEach, Tag} import scala.collection.Seq @@ -65,15 +66,15 @@ abstract class AbstractLogCleanerIntegrationTest { segmentSize: Int = defaultSegmentSize, maxCompactionLagMs: Long = defaultMaxCompactionLagMs): Properties = { val props = new Properties() - props.put(LogConfig.MaxMessageBytesProp, maxMessageSize: java.lang.Integer) - props.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer) - props.put(LogConfig.SegmentIndexBytesProp, 100*1024: java.lang.Integer) - props.put(LogConfig.FileDeleteDelayMsProp, deleteDelay: java.lang.Integer) - props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - props.put(LogConfig.MinCleanableDirtyRatioProp, minCleanableDirtyRatio: java.lang.Float) - props.put(LogConfig.MessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString) - props.put(LogConfig.MinCompactionLagMsProp, minCompactionLagMs: java.lang.Long) - props.put(LogConfig.MaxCompactionLagMsProp, maxCompactionLagMs: java.lang.Long) + props.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageSize: java.lang.Integer) + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentSize: java.lang.Integer) + props.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 100*1024: java.lang.Integer) + props.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, deleteDelay: java.lang.Integer) + props.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + props.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, minCleanableDirtyRatio: java.lang.Float) + props.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.MaxValue.toString) + props.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, minCompactionLagMs: java.lang.Long) + props.put(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, maxCompactionLagMs: java.lang.Long) props ++= propertyOverrides props } @@ -81,7 +82,7 @@ abstract class AbstractLogCleanerIntegrationTest { def makeCleaner(partitions: Iterable[TopicPartition], minCleanableDirtyRatio: Float = defaultMinCleanableDirtyRatio, numThreads: Int = 1, - backOffMs: Long = 15000L, + backoffMs: Long = 15000L, maxMessageSize: Int = defaultMaxMessageSize, minCompactionLagMs: Long = defaultMinCompactionLagMS, deleteDelay: Int = defaultDeleteDelay, @@ -95,7 +96,7 @@ abstract class AbstractLogCleanerIntegrationTest { val dir = new File(logDir, s"${partition.topic}-${partition.partition}") Files.createDirectories(dir.toPath) - val logConfig = LogConfig(logConfigProperties(propertyOverrides, + val logConfig = new LogConfig(logConfigProperties(propertyOverrides, maxMessageSize = maxMessageSize, minCleanableDirtyRatio = minCleanableDirtyRatio, minCompactionLagMs = minCompactionLagMs, @@ -120,11 +121,15 @@ abstract class AbstractLogCleanerIntegrationTest { this.logs += log } - val cleanerConfig = CleanerConfig( - numThreads = numThreads, - ioBufferSize = cleanerIoBufferSize.getOrElse(maxMessageSize / 2), - maxMessageSize = maxMessageSize, - backOffMs = backOffMs) + val cleanerConfig = new CleanerConfig( + numThreads, + 4 * 1024 * 1024L, + 0.9, + cleanerIoBufferSize.getOrElse(maxMessageSize / 2), + maxMessageSize, + Double.MaxValue, + backoffMs, + true) new LogCleaner(cleanerConfig, logDirs = Array(logDir), logs = logMap, diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala index b2daa24962dba..df77aaaed5fde 100755 --- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -19,9 +19,10 @@ package kafka.log import kafka.server.{BrokerTopicStats, FetchLogEnd} import kafka.utils._ +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{LogConfig, LogDirFailureChannel} import org.apache.kafka.server.record.BrokerCompressionType import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api._ @@ -35,7 +36,7 @@ class BrokerCompressionTest { val tmpDir = TestUtils.tempDir() val logDir = TestUtils.randomPartitionLogDir(tmpDir) val time = new MockTime(0, 0) - val logConfig = LogConfig() + val logConfig = new LogConfig(new Properties) @AfterEach def tearDown(): Unit = { @@ -50,11 +51,11 @@ class BrokerCompressionTest { def testBrokerSideCompression(messageCompression: String, brokerCompression: String): Unit = { val messageCompressionType = CompressionType.forName(messageCompression) val logProps = new Properties() - logProps.put(LogConfig.CompressionTypeProp, brokerCompression) + logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, brokerCompression) /*configure broker-side compression */ val log = UnifiedLog( dir = logDir, - config = LogConfig(logProps), + config = new LogConfig(logProps), logStartOffset = 0L, recoveryPoint = 0L, scheduler = time.scheduler, diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala index 6fe947ef57842..9aed8f0f0c4a1 100644 --- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord} import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.server.log.internals.{LogDirFailureChannel, LogOffsetMetadata} +import org.apache.kafka.server.log.internals.{LogConfig, LogDirFailureChannel, LogOffsetMetadata} import org.junit.jupiter.api.Assertions.{assertFalse, _} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 7cfa764d379ff..ae09f7633cc7f 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -51,7 +51,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K val largeMessageKey = 20 val (_, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec) val maxMessageSize = largeMessageSet.sizeInBytes - cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, backOffMs = 100) + cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, backoffMs = 100) def breakPartitionLog(tp: TopicPartition): Unit = { val log = cleaner.logs.get(tp) @@ -134,7 +134,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K val minCleanableDirtyRatio = 1.0F cleaner = makeCleaner(partitions = topicPartitions, - backOffMs = cleanerBackOffMs, + backoffMs = cleanerBackOffMs, minCompactionLagMs = minCompactionLagMs, segmentSize = segmentSize, maxCompactionLagMs= maxCompactionLagMs, @@ -217,7 +217,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest with K @Test def testIsThreadFailed(): Unit = { val metricName = "DeadThreadCount" - cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = 100000, backOffMs = 100) + cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = 100000, backoffMs = 100) cleaner.startup() assertEquals(0, cleaner.deadThreadCount) // we simulate the unexpected error with an interrupt diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala index 7e0a33d132582..9256f30c19d49 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala @@ -46,7 +46,7 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit @MethodSource(Array("parameters")) def cleanerTest(codec: CompressionType): Unit = { cleaner = makeCleaner(partitions = topicPartitions, - backOffMs = cleanerBackOffMs, + backoffMs = cleanerBackOffMs, minCompactionLagMs = minCompactionLag, segmentSize = segmentSize) val log = cleaner.logs.get(topicPartitions(0)) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index a2176d290c2f9..fabf40aa87f35 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -20,12 +20,13 @@ package kafka.log import java.io.File import java.nio.file.Files import java.util.Properties -import kafka.server.{BrokerTopicStats} +import kafka.server.BrokerTopicStats import kafka.utils._ import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -43,10 +44,10 @@ class LogCleanerManagerTest extends Logging { val topicPartition = new TopicPartition("log", 0) val topicPartition2 = new TopicPartition("log2", 0) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - val logConfig: LogConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + val logConfig: LogConfig = new LogConfig(logProps) val time = new MockTime(1400000000000L, 1000L) // Tue May 13 16:53:20 UTC 2014 for `currentTimeMs` val offset = 999 val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs) @@ -81,7 +82,7 @@ class LogCleanerManagerTest extends Logging { var numBatches = startNumBatches for (tp <- partitions) { - val log = createLog(2048, LogConfig.Compact, topicPartition = tp) + val log = createLog(2048, TopicConfig.CLEANUP_POLICY_COMPACT, topicPartition = tp) logs.put(tp, log) writeRecords(log, numBatches = numBatches, recordsPerBatch = 1, batchesPerSegment = 5) @@ -98,7 +99,7 @@ class LogCleanerManagerTest extends Logging { val tpDir = new File(logDir, "A-1") Files.createDirectories(tpDir.toPath) val logDirFailureChannel = new LogDirFailureChannel(10) - val config = createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact) + val config = createLowRetentionLogConfig(logSegmentSize, TopicConfig.CLEANUP_POLICY_COMPACT) val maxTransactionTimeoutMs = 5 * 60 * 1000 val producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs val segments = new LogSegments(tp) @@ -248,7 +249,7 @@ class LogCleanerManagerTest extends Logging { @Test def testLogStartOffsetLargerThanActiveSegmentBaseOffset(): Unit = { val tp = new TopicPartition("foo", 0) - val log = createLog(segmentSize = 2048, LogConfig.Compact, tp) + val log = createLog(segmentSize = 2048, TopicConfig.CLEANUP_POLICY_COMPACT, tp) val logs = new Pool[TopicPartition, UnifiedLog]() logs.put(tp, log) @@ -278,7 +279,7 @@ class LogCleanerManagerTest extends Logging { val tp = new TopicPartition("foo", 0) val logs = new Pool[TopicPartition, UnifiedLog]() - val log = createLog(2048, LogConfig.Compact, topicPartition = tp) + val log = createLog(2048, TopicConfig.CLEANUP_POLICY_COMPACT, topicPartition = tp) logs.put(tp, log) appendRecords(log, numRecords = 3) @@ -303,7 +304,7 @@ class LogCleanerManagerTest extends Logging { @Test def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyDeleteLogs(): Unit = { val records = TestUtils.singletonRecords("test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Delete) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_DELETE) val cleanerManager: LogCleanerManager = createCleanerManager(log) val readyToDelete = cleanerManager.deletableLogs().size @@ -316,7 +317,7 @@ class LogCleanerManagerTest extends Logging { @Test def testLogsWithSegmentsToDeleteShouldConsiderCleanupPolicyCompactDeleteLogs(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val cleanerManager: LogCleanerManager = createCleanerManager(log) val readyToDelete = cleanerManager.deletableLogs().size @@ -330,7 +331,7 @@ class LogCleanerManagerTest extends Logging { @Test def testLogsWithSegmentsToDeleteShouldConsiderCleanupPolicyCompactLogs(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT) val cleanerManager: LogCleanerManager = createCleanerManager(log) val readyToDelete = cleanerManager.deletableLogs().size @@ -343,7 +344,7 @@ class LogCleanerManagerTest extends Logging { @Test def testLogsUnderCleanupIneligibleForCompaction(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Delete) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_DELETE) val cleanerManager: LogCleanerManager = createCleanerManager(log) log.appendAsLeader(records, leaderEpoch = 0) @@ -357,11 +358,11 @@ class LogCleanerManagerTest extends Logging { // change cleanup policy from delete to compact val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, log.config.segmentSize) - logProps.put(LogConfig.RetentionMsProp, log.config.retentionMs) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - logProps.put(LogConfig.MinCleanableDirtyRatioProp, 0: Integer) - val config = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, log.config.segmentSize: Integer) + logProps.put(TopicConfig.RETENTION_MS_CONFIG, log.config.retentionMs: java.lang.Long) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + logProps.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, 0: Integer) + val config = new LogConfig(logProps) log.updateConfig(config) // log cleanup inprogress, the log is not available for compaction @@ -374,8 +375,8 @@ class LogCleanerManagerTest extends Logging { assertEquals(1, cleanable2.size, "should have 1 logs ready to be compacted") // update cleanup policy to delete - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Delete) - val config2 = LogConfig(logProps) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) + val config2 = new LogConfig(logProps) log.updateConfig(config2) // compaction in progress, should have 0 log eligible for log cleanup @@ -391,7 +392,7 @@ class LogCleanerManagerTest extends Logging { @Test def testUpdateCheckpointsShouldAddOffsetToPartition(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT) val cleanerManager: LogCleanerManager = createCleanerManager(log) // expect the checkpoint offset is not the expectedOffset before doing updateCheckpoints @@ -405,7 +406,7 @@ class LogCleanerManagerTest extends Logging { @Test def testUpdateCheckpointsShouldRemovePartitionData(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT) val cleanerManager: LogCleanerManager = createCleanerManager(log) // write some data into the cleaner-offset-checkpoint file @@ -420,7 +421,7 @@ class LogCleanerManagerTest extends Logging { @Test def testHandleLogDirFailureShouldRemoveDirAndData(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT) val cleanerManager: LogCleanerManager = createCleanerManager(log) // write some data into the cleaner-offset-checkpoint file in logDir and logDir2 @@ -438,7 +439,7 @@ class LogCleanerManagerTest extends Logging { @Test def testMaybeTruncateCheckpointShouldTruncateData(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT) val cleanerManager: LogCleanerManager = createCleanerManager(log) val lowerOffset = 1L val higherOffset = 1000L @@ -458,7 +459,7 @@ class LogCleanerManagerTest extends Logging { @Test def testAlterCheckpointDirShouldRemoveDataInSrcDirAndAddInNewDir(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT) val cleanerManager: LogCleanerManager = createCleanerManager(log) // write some data into the cleaner-offset-checkpoint file in logDir @@ -481,7 +482,7 @@ class LogCleanerManagerTest extends Logging { @Test def testConcurrentLogCleanupAndLogTruncation(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Delete) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_DELETE) val cleanerManager: LogCleanerManager = createCleanerManager(log) // log cleanup starts @@ -501,7 +502,7 @@ class LogCleanerManagerTest extends Logging { @Test def testConcurrentLogCleanupAndTopicDeletion(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Delete) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_DELETE) val cleanerManager: LogCleanerManager = createCleanerManager(log) // log cleanup starts @@ -521,7 +522,7 @@ class LogCleanerManagerTest extends Logging { @Test def testLogsWithSegmentsToDeleteShouldNotConsiderUncleanablePartitions(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT) val cleanerManager: LogCleanerManager = createCleanerManager(log) cleanerManager.markPartitionUncleanable(log.dir.getParent, topicPartition) @@ -535,7 +536,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsForNone(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -557,7 +558,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsActiveSegment(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -579,8 +580,8 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.MinCompactionLagMsProp, compactionLag: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, compactionLag: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -612,8 +613,8 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForShortTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.MinCompactionLagMsProp, compactionLag: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, compactionLag: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -654,8 +655,8 @@ class LogCleanerManagerTest extends Logging { def testUndecidedTransactionalDataNotCleanable(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.MinCompactionLagMsProp, compactionLag: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, compactionLag: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -698,7 +699,7 @@ class LogCleanerManagerTest extends Logging { @Test def testDoneCleaning(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) while(log.numberOfSegments < 8) log.appendAsLeader(records(log.logEndOffset.toInt, log.logEndOffset.toInt, time.milliseconds()), leaderEpoch = 0) @@ -726,7 +727,7 @@ class LogCleanerManagerTest extends Logging { @Test def testDoneDeleting(): Unit = { val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes) - val log: UnifiedLog = createLog(records.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete) + val log: UnifiedLog = createLog(records.sizeInBytes * 5, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val cleanerManager: LogCleanerManager = createCleanerManager(log) val tp = new TopicPartition("log", 0) @@ -817,12 +818,12 @@ class LogCleanerManagerTest extends Logging { private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, segmentSize: Integer) - logProps.put(LogConfig.RetentionMsProp, 1: Integer) - logProps.put(LogConfig.CleanupPolicyProp, cleanupPolicy) - logProps.put(LogConfig.MinCleanableDirtyRatioProp, 0.05: java.lang.Double) // small for easier and clearer tests + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentSize: Integer) + logProps.put(TopicConfig.RETENTION_MS_CONFIG, 1: Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, cleanupPolicy) + logProps.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, 0.05: java.lang.Double) // small for easier and clearer tests - LogConfig(logProps) + new LogConfig(logProps) } private def writeRecords(log: UnifiedLog, diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 4229962b5c4bd..f4ae0ad4bee6a 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -19,13 +19,14 @@ package kafka.log import java.io.File import java.util.Properties - import kafka.server.KafkaConfig import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils._ import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ -import org.apache.kafka.server.common.MetadataVersion.{IBP_0_9_0, IBP_0_10_0_IV1, IBP_0_11_0_IV0} +import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0} +import org.apache.kafka.server.log.internals.{CleanerConfig, LogConfig} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.extension.ExtensionContext import org.junit.jupiter.params.ParameterizedTest @@ -92,11 +93,11 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati def testCleansCombinedCompactAndDeleteTopic(codec: CompressionType): Unit = { val logProps = new Properties() val retentionMs: Integer = 100000 - logProps.put(LogConfig.RetentionMsProp, retentionMs: Integer) - logProps.put(LogConfig.CleanupPolicyProp, "compact,delete") + logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact,delete") def runCleanerAndCheckCompacted(numKeys: Int): (UnifiedLog, Seq[(Int, String, Long)]) = { - cleaner = makeCleaner(partitions = topicPartitions.take(1), propertyOverrides = logProps, backOffMs = 100L) + cleaner = makeCleaner(partitions = topicPartitions.take(1), propertyOverrides = logProps, backoffMs = 100L) val log = cleaner.logs.get(topicPartitions(0)) val messages = writeDups(numKeys = numKeys, numDups = 3, log = log, codec = codec) @@ -151,7 +152,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val log = cleaner.logs.get(topicPartitions(0)) val props = logConfigProperties(maxMessageSize = maxMessageSize) - props.put(LogConfig.MessageFormatVersionProp, IBP_0_9_0.version) + props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_9_0.version) log.updateConfig(new LogConfig(props)) val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0) @@ -173,7 +174,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val largeMessageOffset = appendInfo.firstOffset.map(_.messageOffset).get // also add some messages with version 1 and version 2 to check that we handle mixed format versions correctly - props.put(LogConfig.MessageFormatVersionProp, IBP_0_11_0_IV0.version) + props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_11_0_IV0.version) log.updateConfig(new LogConfig(props)) val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) val dupsV2 = writeDups(startKey = 15, numKeys = 5, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V2) @@ -194,7 +195,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val log = cleaner.logs.get(topicPartitions(0)) val props = logConfigProperties(maxMessageSize = maxMessageSize, segmentSize = 256) - props.put(LogConfig.MessageFormatVersionProp, IBP_0_9_0.version) + props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_9_0.version) log.updateConfig(new LogConfig(props)) // with compression enabled, these messages will be written as a single message containing @@ -202,7 +203,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0) appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0) - props.put(LogConfig.MessageFormatVersionProp, IBP_0_10_0_IV1.version) + props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_10_0_IV1.version) log.updateConfig(new LogConfig(props)) var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) @@ -231,7 +232,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val (_, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec) val maxMessageSize = largeMessageSet.sizeInBytes - cleaner = makeCleaner(partitions = topicPartitions, backOffMs = 1, maxMessageSize = maxMessageSize, + cleaner = makeCleaner(partitions = topicPartitions, backoffMs = 1, maxMessageSize = maxMessageSize, cleanerIoBufferSize = Some(1)) val log = cleaner.logs.get(topicPartitions(0)) @@ -253,20 +254,21 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati props.put(KafkaConfig.LogCleanerDedupeBufferLoadFactorProp, cleanerConfig.dedupeBufferLoadFactor.toString) props.put(KafkaConfig.LogCleanerIoBufferSizeProp, cleanerConfig.ioBufferSize.toString) props.put(KafkaConfig.MessageMaxBytesProp, cleanerConfig.maxMessageSize.toString) - props.put(KafkaConfig.LogCleanerBackoffMsProp, cleanerConfig.backOffMs.toString) + props.put(KafkaConfig.LogCleanerBackoffMsProp, cleanerConfig.backoffMs.toString) props.put(KafkaConfig.LogCleanerIoMaxBytesPerSecondProp, cleanerConfig.maxIoBytesPerSecond.toString) KafkaConfig.fromProps(props) } // Verify cleaning done with larger LogCleanerIoBufferSizeProp val oldConfig = kafkaConfigWithCleanerConfig(cleaner.currentConfig) - val newConfig = kafkaConfigWithCleanerConfig(CleanerConfig(numThreads = 2, - dedupeBufferSize = cleaner.currentConfig.dedupeBufferSize, - dedupeBufferLoadFactor = cleaner.currentConfig.dedupeBufferLoadFactor, - ioBufferSize = 100000, - maxMessageSize = cleaner.currentConfig.maxMessageSize, - maxIoBytesPerSecond = cleaner.currentConfig.maxIoBytesPerSecond, - backOffMs = cleaner.currentConfig.backOffMs)) + val newConfig = kafkaConfigWithCleanerConfig(new CleanerConfig(2, + cleaner.currentConfig.dedupeBufferSize, + cleaner.currentConfig.dedupeBufferLoadFactor, + 100000, + cleaner.currentConfig.maxMessageSize, + cleaner.currentConfig.maxIoBytesPerSecond, + cleaner.currentConfig.backoffMs, + true)) cleaner.reconfigure(oldConfig, newConfig) assertEquals(2, cleaner.cleanerCount) @@ -277,7 +279,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati private def checkLastCleaned(topic: String, partitionId: Int, firstDirty: Long): Unit = { // wait until cleaning up to base_offset, note that cleaning happens only when "log dirty ratio" is higher than - // LogConfig.MinCleanableDirtyRatioProp + // TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG val topicPartition = new TopicPartition(topic, partitionId) cleaner.awaitCleaned(topicPartition, firstDirty) val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints(topicPartition) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 8cdd472c632e4..54b0f131ffc1e 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -27,10 +27,11 @@ import kafka.common._ import kafka.server.{BrokerTopicStats, KafkaConfig} import kafka.utils._ import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LogDirFailureChannel, OffsetMap} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CleanerConfig, LogConfig, LogDirFailureChannel, OffsetMap} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -45,11 +46,11 @@ class LogCleanerTest { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString) - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.MaxValue.toString) + val logConfig = new LogConfig(logProps) val time = new MockTime() val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) val tombstoneRetentionMs = 86400000 @@ -68,7 +69,7 @@ class LogCleanerTest { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -101,8 +102,8 @@ class LogCleanerTest { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024 : java.lang.Integer) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact + "," + LogConfig.Delete) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) val logDirFailureChannel = new LogDirFailureChannel(10) @@ -187,9 +188,9 @@ class LogCleanerTest { val originalMaxFileSize = 1024; val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, originalMaxFileSize: java.lang.Integer) - logProps.put(LogConfig.CleanupPolicyProp, "compact": java.lang.String) - logProps.put(LogConfig.PreAllocateEnableProp, "true": java.lang.String) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) + logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0,0), leaderEpoch = 0) // offset 0 @@ -211,7 +212,7 @@ class LogCleanerTest { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -292,7 +293,7 @@ class LogCleanerTest { val offsetMapSlots = 4 val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.DeleteRetentionMsProp, deleteRetentionMs.toString) + logProps.put(TopicConfig.DELETE_RETENTION_MS_CONFIG, deleteRetentionMs.toString) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -380,7 +381,7 @@ class LogCleanerTest { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -413,7 +414,7 @@ class LogCleanerTest { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -460,7 +461,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -508,8 +509,8 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() - logProps.put(LogConfig.MaxMessageBytesProp, 100: java.lang.Integer) - logProps.put(LogConfig.SegmentBytesProp, 1000: java.lang.Integer) + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -538,7 +539,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -605,7 +606,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -639,7 +640,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -662,7 +663,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -692,7 +693,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -727,7 +728,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, @@ -763,7 +764,7 @@ class LogCleanerTest { val tp = new TopicPartition("test", 0) val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -827,8 +828,8 @@ class LogCleanerTest { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, largeMessageSize * 16: java.lang.Integer) - logProps.put(LogConfig.MaxMessageBytesProp, largeMessageSize * 2: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -899,8 +900,8 @@ class LogCleanerTest { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, largeMessageSize * 16: java.lang.Integer) - logProps.put(LogConfig.MaxMessageBytesProp, largeMessageSize * 2: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -910,7 +911,7 @@ class LogCleanerTest { assertEquals(0L until log.logEndOffset, keysFound) // Decrease the log's max message size - logProps.put(LogConfig.MaxMessageBytesProp, largeMessageSize / 2: java.lang.Integer) + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize / 2: java.lang.Integer) log.updateConfig(LogConfig.fromProps(logConfig.originals, logProps)) // pretend we have the following keys @@ -925,7 +926,7 @@ class LogCleanerTest { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -952,7 +953,7 @@ class LogCleanerTest { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -980,7 +981,7 @@ class LogCleanerTest { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), leaderEpoch = 0) // offset 0 @@ -1003,7 +1004,7 @@ class LogCleanerTest { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1037,7 +1038,7 @@ class LogCleanerTest { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1081,7 +1082,7 @@ class LogCleanerTest { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1112,7 +1113,7 @@ class LogCleanerTest { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1155,7 +1156,7 @@ class LogCleanerTest { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1173,7 +1174,7 @@ class LogCleanerTest { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1206,8 +1207,8 @@ class LogCleanerTest { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Delete) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1264,7 +1265,7 @@ class LogCleanerTest { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1288,8 +1289,8 @@ class LogCleanerTest { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) - logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1389,8 +1390,8 @@ class LogCleanerTest { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 400: java.lang.Integer) - logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1441,10 +1442,10 @@ class LogCleanerTest { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 400: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file - logProps.put(LogConfig.IndexIntervalBytesProp, 400: java.lang.Integer) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1512,8 +1513,8 @@ class LogCleanerTest { def testSegmentWithOffsetOverflow(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) - logProps.put(LogConfig.FileDeleteDelayMsProp, 1000: java.lang.Integer) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) + logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 1000: java.lang.Integer) val config = LogConfig.fromProps(logConfig.originals, logProps) LogTestUtils.initializeLogDirWithOverflowedSegment(dir) @@ -1565,9 +1566,9 @@ class LogCleanerTest { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 300: java.lang.Integer) - logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) - logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) + logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) val config = LogConfig.fromProps(logConfig.originals, logProps) @@ -1694,10 +1695,10 @@ class LogCleanerTest { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 120: java.lang.Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, 120: java.lang.Integer) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + val logConfig = new LogConfig(logProps) val log = makeLog(config = logConfig) val cleaner = makeCleaner(Int.MaxValue) val keyStart = 0 @@ -1748,8 +1749,8 @@ class LogCleanerTest { val codec = CompressionType.GZIP val logProps = new Properties() - logProps.put(LogConfig.CompressionTypeProp, codec.name) - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, codec.name) + val logConfig = new LogConfig(logProps) val log = makeLog(config = logConfig) val cleaner = makeCleaner(10) @@ -1803,7 +1804,7 @@ class LogCleanerTest { @Test def testCleanTombstone(): Unit = { - val logConfig = LogConfig(new Properties()) + val logConfig = new LogConfig(new Properties()) val log = makeLog(config = logConfig) val cleaner = makeCleaner(10) @@ -1839,9 +1840,9 @@ class LogCleanerTest { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024*1024: java.lang.Integer) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) + val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) { @@ -1880,7 +1881,7 @@ class LogCleanerTest { @Test def testMaxCleanTimeSecs(): Unit = { - val logCleaner = new LogCleaner(new CleanerConfig, + val logCleaner = new LogCleaner(new CleanerConfig(true), logDirs = Array(TestUtils.tempDir()), logs = new Pool[TopicPartition, UnifiedLog](), logDirFailureChannel = new LogDirFailureChannel(1), diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 14a23f68e09f4..318a899ced254 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -19,12 +19,12 @@ package kafka.log import java.util.Properties import java.util.concurrent.{Callable, Executors} - import kafka.server.{BrokerTopicStats, FetchHighWatermark} import kafka.utils.{KafkaScheduler, TestUtils} +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{LogConfig, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -57,8 +57,8 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 237: Integer) - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) + val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } @@ -141,7 +141,7 @@ class LogConcurrencyTest { } } - private def createLog(config: LogConfig = LogConfig(new Properties())): UnifiedLog = { + private def createLog(config: LogConfig = new LogConfig(new Properties())): UnifiedLog = { UnifiedLog(dir = logDir, config = config, logStartOffset = 0L, diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 838c043ff8426..5389b7cf26fca 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -17,18 +17,20 @@ package kafka.log -import kafka.server.{KafkaConfig, ThrottledReplicaListValidator} +import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM import org.apache.kafka.common.config.ConfigDef.Type.INT import org.apache.kafka.common.config.{ConfigException, TopicConfig} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test -import java.util.{Collections, Properties} +import java.util.{Collections, Properties} import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1 +import org.apache.kafka.server.log.internals.{LogConfig, ThrottledReplicaListValidator} import scala.annotation.nowarn +import scala.jdk.CollectionConverters._ class LogConfigTest { @@ -43,10 +45,11 @@ class LogConfigTest { def ensureNoStaticInitializationOrderDependency(): Unit = { // Access any KafkaConfig val to load KafkaConfig object before LogConfig. assertNotNull(KafkaConfig.LogRetentionTimeMillisProp) - assertTrue(LogConfig.configNames.filter(config => !LogConfig.configsWithNoServerDefaults.contains(config)) + assertTrue(LogConfig.configNames.asScala + .filter(config => !LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS.contains(config)) .forall { config => val serverConfigOpt = LogConfig.serverConfigName(config) - serverConfigOpt.isDefined && (serverConfigOpt.get != null) + serverConfigOpt.isPresent && (serverConfigOpt.get != null) }) } @@ -60,36 +63,28 @@ class LogConfigTest { kafkaProps.put(KafkaConfig.LogRetentionTimeHoursProp, "2") kafkaProps.put(KafkaConfig.LogMessageFormatVersionProp, "0.11.0") - val kafkaConfig = KafkaConfig.fromProps(kafkaProps) - val logProps = LogConfig.extractLogConfigMap(kafkaConfig) - assertEquals(2 * millisInHour, logProps.get(LogConfig.SegmentMsProp)) - assertEquals(2 * millisInHour, logProps.get(LogConfig.SegmentJitterMsProp)) - assertEquals(2 * millisInHour, logProps.get(LogConfig.RetentionMsProp)) + val logProps = KafkaConfig.fromProps(kafkaProps).extractLogConfigMap + assertEquals(2 * millisInHour, logProps.get(TopicConfig.SEGMENT_MS_CONFIG)) + assertEquals(2 * millisInHour, logProps.get(TopicConfig.SEGMENT_JITTER_MS_CONFIG)) + assertEquals(2 * millisInHour, logProps.get(TopicConfig.RETENTION_MS_CONFIG)) // The message format version should always be 3.0 if the inter-broker protocol version is 3.0 or higher - assertEquals(IBP_3_0_IV1.version, logProps.get(LogConfig.MessageFormatVersionProp)) - } - - @Test - def testFromPropsEmpty(): Unit = { - val p = new Properties() - val config = LogConfig(p) - assertEquals(LogConfig(), config) + assertEquals(IBP_3_0_IV1.version, logProps.get(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) } @nowarn("cat=deprecation") @Test def testFromPropsInvalid(): Unit = { - LogConfig.configNames.foreach(name => name match { - case LogConfig.UncleanLeaderElectionEnableProp => assertPropertyInvalid(name, "not a boolean") - case LogConfig.RetentionBytesProp => assertPropertyInvalid(name, "not_a_number") - case LogConfig.RetentionMsProp => assertPropertyInvalid(name, "not_a_number" ) - case LogConfig.CleanupPolicyProp => assertPropertyInvalid(name, "true", "foobar") - case LogConfig.MinCleanableDirtyRatioProp => assertPropertyInvalid(name, "not_a_number", "-0.1", "1.2") - case LogConfig.MinInSyncReplicasProp => assertPropertyInvalid(name, "not_a_number", "0", "-1") - case LogConfig.MessageFormatVersionProp => assertPropertyInvalid(name, "") - case LogConfig.RemoteLogStorageEnableProp => assertPropertyInvalid(name, "not_a_boolean") - case LogConfig.LocalLogRetentionMsProp => assertPropertyInvalid(name, "not_a_number", "-3") - case LogConfig.LocalLogRetentionBytesProp => assertPropertyInvalid(name, "not_a_number", "-3") + LogConfig.configNames.forEach(name => name match { + case TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG => assertPropertyInvalid(name, "not a boolean") + case TopicConfig.RETENTION_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number") + case TopicConfig.RETENTION_MS_CONFIG => assertPropertyInvalid(name, "not_a_number" ) + case TopicConfig.CLEANUP_POLICY_CONFIG => assertPropertyInvalid(name, "true", "foobar") + case TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1", "1.2") + case TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG => assertPropertyInvalid(name, "not_a_number", "0", "-1") + case TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG => assertPropertyInvalid(name, "") + case TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG => assertPropertyInvalid(name, "not_a_boolean") + case TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") + case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) @@ -98,8 +93,8 @@ class LogConfigTest { @Test def testInvalidCompactionLagConfig(): Unit = { val props = new Properties - props.setProperty(LogConfig.MaxCompactionLagMsProp, "100") - props.setProperty(LogConfig.MinCompactionLagMsProp, "200") + props.setProperty(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, "100") + props.setProperty(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, "200") assertThrows(classOf[Exception], () => LogConfig.validate(props)) } @@ -167,11 +162,11 @@ class LogConfigTest { configDef.define(configNameWithNoServerMapping, INT, 1, MEDIUM, s"$configNameWithNoServerMapping doc") val deleteDelayKey = configDef.configKeys.get(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG) - val deleteDelayServerDefault = configDef.getConfigValue(deleteDelayKey, LogConfig.ServerDefaultHeaderName) + val deleteDelayServerDefault = configDef.getConfigValue(deleteDelayKey, LogConfig.SERVER_DEFAULT_HEADER_NAME) assertEquals(KafkaConfig.LogDeleteDelayMsProp, deleteDelayServerDefault) val keyWithNoServerMapping = configDef.configKeys.get(configNameWithNoServerMapping) - val nullServerDefault = configDef.getConfigValue(keyWithNoServerMapping, LogConfig.ServerDefaultHeaderName) + val nullServerDefault = configDef.getConfigValue(keyWithNoServerMapping, LogConfig.SERVER_DEFAULT_HEADER_NAME) assertNull(nullServerDefault) } @@ -184,15 +179,15 @@ class LogConfigTest { val kafkaConfig = KafkaConfig.fromProps(kafkaProps) val topicOverrides = new Properties // Only set as a topic config - topicOverrides.setProperty(LogConfig.MinInSyncReplicasProp, "2") + topicOverrides.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2") // Overrides value from broker config - topicOverrides.setProperty(LogConfig.RetentionBytesProp, "100") + topicOverrides.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "100") // Unknown topic config, but known broker config topicOverrides.setProperty(KafkaConfig.SslTruststorePasswordProp, "sometrustpasswrd") // Unknown config topicOverrides.setProperty("unknown.topic.password.config", "bbbb") // We don't currently have any sensitive topic configs, if we add them, we should set one here - val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(kafkaConfig), topicOverrides) + val logConfig = LogConfig.fromProps(kafkaConfig.extractLogConfigMap, topicOverrides) assertEquals("{min.insync.replicas=2, retention.bytes=100, ssl.truststore.password=(redacted), unknown.topic.password.config=(redacted)}", logConfig.overriddenConfigsAsLoggableString) } @@ -210,7 +205,7 @@ class LogConfigTest { values.foreach((value) => { val props = new Properties props.setProperty(name, value.toString) - assertThrows(classOf[Exception], () => LogConfig(props)) + assertThrows(classOf[Exception], () => new LogConfig(props)) }) } @@ -219,8 +214,8 @@ class LogConfigTest { val props = new Properties() val retentionBytes = 1024 val retentionMs = 1000L - props.put(LogConfig.RetentionBytesProp, retentionBytes.toString) - props.put(LogConfig.RetentionMsProp, retentionMs.toString) + props.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes.toString) + props.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs.toString) val logConfig = new LogConfig(props) assertEquals(retentionMs, logConfig.remoteLogConfig.localRetentionMs) @@ -229,11 +224,11 @@ class LogConfigTest { @Test def testLocalLogRetentionDerivedDefaultProps(): Unit = { - val logConfig = new LogConfig( new Properties()) + val logConfig = new LogConfig(new Properties()) // Local retention defaults are derived from retention properties which can be default or custom. - assertEquals(Defaults.RetentionMs, logConfig.remoteLogConfig.localRetentionMs) - assertEquals(Defaults.RetentionSize, logConfig.remoteLogConfig.localRetentionBytes) + assertEquals(LogConfig.DEFAULT_RETENTION_MS, logConfig.remoteLogConfig.localRetentionMs) + assertEquals(LogConfig.DEFAULT_RETENTION_BYTES, logConfig.remoteLogConfig.localRetentionBytes) } @Test @@ -241,11 +236,11 @@ class LogConfigTest { val props = new Properties() val localRetentionMs = 500 val localRetentionBytes = 1000 - props.put(LogConfig.RetentionBytesProp, 2000.toString) - props.put(LogConfig.RetentionMsProp, 1000.toString) + props.put(TopicConfig.RETENTION_BYTES_CONFIG, 2000.toString) + props.put(TopicConfig.RETENTION_MS_CONFIG, 1000.toString) - props.put(LogConfig.LocalLogRetentionMsProp, localRetentionMs.toString) - props.put(LogConfig.LocalLogRetentionBytesProp, localRetentionBytes.toString) + props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs.toString) + props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localRetentionBytes.toString) val logConfig = new LogConfig(props) assertEquals(localRetentionMs, logConfig.remoteLogConfig.localRetentionMs) @@ -275,11 +270,11 @@ class LogConfigTest { private def doTestInvalidLocalLogRetentionProps(localRetentionMs: Long, localRetentionBytes: Int, retentionBytes: Int, retentionMs: Long) = { val props = new Properties() - props.put(LogConfig.RetentionBytesProp, retentionBytes.toString) - props.put(LogConfig.RetentionMsProp, retentionMs.toString) + props.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes.toString) + props.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs.toString) - props.put(LogConfig.LocalLogRetentionMsProp, localRetentionMs.toString) - props.put(LogConfig.LocalLogRetentionBytesProp, localRetentionBytes.toString) + props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs.toString) + props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localRetentionBytes.toString) assertThrows(classOf[ConfigException], () => new LogConfig(props)); } } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 0a46c5965fa56..803f6ea9d350a 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -26,12 +26,13 @@ import kafka.server.{BrokerTopicStats, FetchDataInfo, KafkaConfig} import kafka.server.metadata.MockConfigRepository import kafka.utils.{MockTime, Scheduler, TestUtils} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{CompressionType, ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0 -import org.apache.kafka.server.log.internals.{AbortedTxn, LogDirFailureChannel, OffsetIndex, SnapshotFile} +import org.apache.kafka.server.log.internals.{AbortedTxn, CleanerConfig, LogConfig, LogDirFailureChannel, OffsetIndex, SnapshotFile} import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue} import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -79,7 +80,7 @@ class LogLoaderTest { // to run recovery while loading upon subsequent broker boot up. val logDir: File = TestUtils.tempDir() val logProps = new Properties() - val logConfig = LogConfig(logProps) + val logConfig = new LogConfig(logProps) val logDirs = Seq(logDir) val topicPartition = new TopicPartition("foo", 0) var log: UnifiedLog = null @@ -103,7 +104,7 @@ class LogLoaderTest { initialOfflineDirs = Array.empty[File], configRepository = new MockConfigRepository(), initialDefaultConfig = logConfig, - cleanerConfig = CleanerConfig(enableCleaner = false), + cleanerConfig = new CleanerConfig(false), recoveryThreadsPerDataDir = 4, flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, @@ -293,9 +294,9 @@ class LogLoaderTest { @nowarn("cat=deprecation") private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, "640") - logProps.put(LogConfig.MessageFormatVersionProp, messageFormatVersion) - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, messageFormatVersion) + val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(None, log.oldestProducerSnapshotOffset) @@ -417,7 +418,7 @@ class LogLoaderTest { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) - val config = LogConfig(new Properties()) + val config = new LogConfig(new Properties()) val segments = new LogSegments(topicPartition) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") val offsets = new LogLoader( @@ -524,8 +525,8 @@ class LogLoaderTest { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val config = LogConfig(logProps) + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2") + val config = new LogConfig(logProps) val logDirFailureChannel = null val segments = new LogSegments(topicPartition) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") @@ -578,8 +579,8 @@ class LogLoaderTest { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val config = LogConfig(logProps) + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2") + val config = new LogConfig(logProps) val logDirFailureChannel = null val segments = new LogSegments(topicPartition) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") @@ -631,8 +632,8 @@ class LogLoaderTest { val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.11.0") - val config = LogConfig(logProps) + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0") + val config = new LogConfig(logProps) val logDirFailureChannel = null val segments = new LogSegments(topicPartition) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.recordVersion, "") @@ -863,10 +864,10 @@ class LogLoaderTest { val numMessages = 200 val segmentSize = 200 val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, segmentSize.toString) - logProps.put(LogConfig.IndexIntervalBytesProp, "1") - logProps.put(LogConfig.MessageFormatVersionProp, "0.9.0") - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentSize.toString) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.9.0") + val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) for (i <- 0 until numMessages) log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), @@ -1091,11 +1092,11 @@ class LogLoaderTest { // reopen the log with an older message format version and check the cache val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, "1000") - logProps.put(LogConfig.IndexIntervalBytesProp, "1") - logProps.put(LogConfig.MaxMessageBytesProp, "65536") - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val downgradedLogConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2") + val downgradedLogConfig = new LogConfig(logProps) val reopened = createLog(logDir, downgradedLogConfig, lastShutdownClean = false) LogTestUtils.assertLeaderEpochCacheEmpty(reopened) @@ -1358,7 +1359,7 @@ class LogLoaderTest { */ @Test def testLogRecoversForLeaderEpoch(): Unit = { - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) val leaderEpochCache = log.leaderEpochCache.get val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0) log.appendAsFollower(records = firstBatch) @@ -1380,7 +1381,7 @@ class LogLoaderTest { log.close() // reopen the log and recover from the beginning - val recoveredLog = createLog(logDir, LogConfig(), lastShutdownClean = false) + val recoveredLog = createLog(logDir, new LogConfig(new Properties), lastShutdownClean = false) val recoveredLeaderEpochCache = recoveredLog.leaderEpochCache.get // epoch entries should be recovered diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 3c17878ae799d..1e75ecbcff53d 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -24,6 +24,7 @@ import kafka.server.metadata.{ConfigRepository, MockConfigRepository} import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchLogEnd} import kafka.utils._ import org.apache.directory.api.util.FileUtils +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.OffsetOutOfRangeException import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{KafkaException, TopicPartition} @@ -37,7 +38,7 @@ import java.io._ import java.nio.file.Files import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} import java.util.{Collections, Properties} -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{LogConfig, LogDirFailureChannel} import org.apache.kafka.server.metrics.KafkaYammerMetrics import scala.collection.{Map, mutable} @@ -51,11 +52,11 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) - logProps.put(LogConfig.RetentionMsProp, maxLogAgeMs: java.lang.Integer) - logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString) - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) + logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) + logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.MaxValue.toString) + val logConfig = new LogConfig(logProps) var logDir: File = _ var logManager: LogManager = _ val name = "kafka" @@ -251,8 +252,8 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(LogConfig.SegmentBytesProp, segmentBytes.toString) - properties.put(LogConfig.RetentionBytesProp, (5L * 10L * setSize + 10L).toString) + properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) logManager = createLogManager(configRepository = configRepository) @@ -293,7 +294,7 @@ class LogManagerTest { */ @Test def testDoesntCleanLogsWithCompactDeletePolicy(): Unit = { - testDoesntCleanLogs(LogConfig.Compact + "," + LogConfig.Delete) + testDoesntCleanLogs(TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) } /** @@ -302,12 +303,12 @@ class LogManagerTest { */ @Test def testDoesntCleanLogsWithCompactPolicy(): Unit = { - testDoesntCleanLogs(LogConfig.Compact) + testDoesntCleanLogs(TopicConfig.CLEANUP_POLICY_COMPACT) } private def testDoesntCleanLogs(policy: String): Unit = { logManager.shutdown() - val configRepository = MockConfigRepository.forTopic(name, LogConfig.CleanupPolicyProp, policy) + val configRepository = MockConfigRepository.forTopic(name, TopicConfig.CLEANUP_POLICY_CONFIG, policy) logManager = createLogManager(configRepository = configRepository) val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = None) @@ -333,7 +334,7 @@ class LogManagerTest { @Test def testTimeBasedFlush(): Unit = { logManager.shutdown() - val configRepository = MockConfigRepository.forTopic(name, LogConfig.FlushMsProp, "1000") + val configRepository = MockConfigRepository.forTopic(name, TopicConfig.FLUSH_MS_CONFIG, "1000") logManager = createLogManager(configRepository = configRepository) logManager.startup(Set.empty) @@ -617,7 +618,7 @@ class LogManagerTest { val tp1 = new TopicPartition(topic, 1) val oldProperties = new Properties() - oldProperties.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + oldProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val oldLogConfig = LogConfig.fromProps(logConfig.originals, oldProperties) val log0 = spyLogManager.getOrCreateLog(tp0, topicId = None) @@ -628,7 +629,7 @@ class LogManagerTest { assertEquals(Set(log0, log1), spyLogManager.logsByTopic(topic).toSet) val newProperties = new Properties() - newProperties.put(LogConfig.CleanupPolicyProp, LogConfig.Delete) + newProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) spyLogManager.updateTopicConfig(topic, newProperties) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index c1776a4344c4a..85b4801a0ece6 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -17,14 +17,15 @@ package kafka.log import java.io.File - import kafka.server.checkpoints.LeaderEpochCheckpoint import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} import kafka.utils.TestUtils import kafka.utils.TestUtils.checkEquals import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{MockTime, Time, Utils} +import org.apache.kafka.server.log.internals.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -466,10 +467,10 @@ class LogSegmentTest { private def createSegment(baseOffset: Long, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean): LogSegment = { val tempDir = TestUtils.tempDir() - val logConfig = LogConfig(Map( - LogConfig.IndexIntervalBytesProp -> 10, - LogConfig.SegmentIndexBytesProp -> 1000, - LogConfig.SegmentJitterMsProp -> 0 + val logConfig = new LogConfig(Map( + TopicConfig.INDEX_INTERVAL_BYTES_CONFIG -> 10, + TopicConfig.SEGMENT_INDEX_BYTES_CONFIG -> 1000, + TopicConfig.SEGMENT_JITTER_MS_CONFIG -> 0 ).asJava) val seg = LogSegment.open(tempDir, baseOffset, logConfig, Time.SYSTEM, fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate) @@ -493,10 +494,10 @@ class LogSegmentTest { @Test def testCreateWithInitFileSizeClearShutdown(): Unit = { val tempDir = TestUtils.tempDir() - val logConfig = LogConfig(Map( - LogConfig.IndexIntervalBytesProp -> 10, - LogConfig.SegmentIndexBytesProp -> 1000, - LogConfig.SegmentJitterMsProp -> 0 + val logConfig = new LogConfig(Map( + TopicConfig.INDEX_INTERVAL_BYTES_CONFIG -> 10, + TopicConfig.SEGMENT_INDEX_BYTES_CONFIG -> 1000, + TopicConfig.SEGMENT_JITTER_MS_CONFIG -> 0 ).asJava) val seg = LogSegment.open(tempDir, baseOffset = 40, logConfig, Time.SYSTEM, diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index b614e11c7e703..7386d4a31bd35 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -27,13 +27,13 @@ import kafka.utils.{Scheduler, TestUtils} import org.apache.kafka.common.Uuid import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LazyIndex, LogConfig, LogDirFailureChannel, TransactionIndex} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import java.nio.file.Files import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import kafka.log -import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LazyIndex, TransactionIndex} +import org.apache.kafka.common.config.TopicConfig import scala.collection.Iterable import scala.jdk.CollectionConverters._ @@ -54,30 +54,30 @@ object LogTestUtils { new LogSegment(ms, idx, timeIdx, txnIndex, offset, indexIntervalBytes, 0, time) } - def createLogConfig(segmentMs: Long = Defaults.SegmentMs, - segmentBytes: Int = Defaults.SegmentSize, - retentionMs: Long = Defaults.RetentionMs, - retentionBytes: Long = Defaults.RetentionSize, - segmentJitterMs: Long = Defaults.SegmentJitterMs, - cleanupPolicy: String = Defaults.CleanupPolicy, - maxMessageBytes: Int = Defaults.MaxMessageSize, - indexIntervalBytes: Int = Defaults.IndexInterval, - segmentIndexBytes: Int = Defaults.MaxIndexSize, - fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs, - remoteLogStorageEnable: Boolean = Defaults.RemoteLogStorageEnable): LogConfig = { + def createLogConfig(segmentMs: Long = LogConfig.DEFAULT_SEGMENT_MS, + segmentBytes: Int = LogConfig.DEFAULT_SEGMENT_BYTES, + retentionMs: Long = LogConfig.DEFAULT_RETENTION_MS, + retentionBytes: Long = LogConfig.DEFAULT_RETENTION_BYTES, + segmentJitterMs: Long = LogConfig.DEFAULT_SEGMENT_JITTER_MS, + cleanupPolicy: String = LogConfig.DEFAULT_CLEANUP_POLICY, + 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, + remoteLogStorageEnable: Boolean = LogConfig.DEFAULT_REMOTE_STORAGE_ENABLE): LogConfig = { val logProps = new Properties() - logProps.put(LogConfig.SegmentMsProp, segmentMs: java.lang.Long) - logProps.put(LogConfig.SegmentBytesProp, segmentBytes: Integer) - logProps.put(LogConfig.RetentionMsProp, retentionMs: java.lang.Long) - logProps.put(LogConfig.RetentionBytesProp, retentionBytes: java.lang.Long) - logProps.put(LogConfig.SegmentJitterMsProp, segmentJitterMs: java.lang.Long) - logProps.put(LogConfig.CleanupPolicyProp, cleanupPolicy) - logProps.put(LogConfig.MaxMessageBytesProp, maxMessageBytes: Integer) - logProps.put(LogConfig.IndexIntervalBytesProp, indexIntervalBytes: Integer) - logProps.put(LogConfig.SegmentIndexBytesProp, segmentIndexBytes: Integer) - logProps.put(LogConfig.FileDeleteDelayMsProp, fileDeleteDelayMs: java.lang.Long) - logProps.put(LogConfig.RemoteLogStorageEnableProp, remoteLogStorageEnable: java.lang.Boolean) - LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) + logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) + logProps.put(TopicConfig.SEGMENT_JITTER_MS_CONFIG, segmentJitterMs: java.lang.Long) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, cleanupPolicy) + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageBytes: Integer) + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, indexIntervalBytes: Integer) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, segmentIndexBytes: Integer) + logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, fileDeleteDelayMs: java.lang.Long) + logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, remoteLogStorageEnable: java.lang.Boolean) + new LogConfig(logProps) } def createLog(dir: File, diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 5db1ae2633a74..bf4e40b39fcf8 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -28,6 +28,7 @@ import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, PartitionMetadataFile} import kafka.utils._ +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.{InvalidRecordException, TopicPartition, Uuid} import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic @@ -37,7 +38,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} -import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LogOffsetMetadata, RecordValidationException} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LogConfig, LogOffsetMetadata, RecordValidationException} import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ @@ -575,7 +576,7 @@ class UnifiedLogTest { @Test def testNonSequentialAppend(): Unit = { // create a log - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) val pid = 1L val epoch: Short = 0 @@ -588,7 +589,7 @@ class UnifiedLogTest { @Test def testTruncateToEndOffsetClearsEpochCache(): Unit = { - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) // Seed some initial data in the log val records = TestUtils.records(List(new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)), @@ -985,7 +986,7 @@ class UnifiedLogTest { @Test def testCompactionDeletesProducerStateSnapshots(): Unit = { - val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, cleanupPolicy = LogConfig.Compact, fileDeleteDelayMs = 0) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT, fileDeleteDelayMs = 0) val log = createLog(logDir, logConfig) val pid1 = 1L val epoch = 0.toShort @@ -1196,7 +1197,7 @@ class UnifiedLogTest { @Test def testDuplicateAppends(): Unit = { // create a log - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) val pid = 1L val epoch: Short = 0 @@ -1272,7 +1273,7 @@ class UnifiedLogTest { @Test def testMultipleProducerIdsPerMemoryRecord(): Unit = { // create a log - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) val epoch: Short = 0 val buffer = ByteBuffer.allocate(512) @@ -1391,7 +1392,7 @@ class UnifiedLogTest { @Test def testOldProducerEpoch(): Unit = { // create a log - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) val pid = 1L val newEpoch: Short = 1 val oldEpoch: Short = 0 @@ -1487,7 +1488,7 @@ class UnifiedLogTest { @Test def testLoadEmptyLog(): Unit = { createEmptyLogs(logDir, 0) - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) log.appendAsLeader(TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds), leaderEpoch = 0) } @@ -1779,7 +1780,7 @@ class UnifiedLogTest { val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage) val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage) - val logConfig = LogTestUtils.createLogConfig(cleanupPolicy = LogConfig.Compact) + val logConfig = LogTestUtils.createLogConfig(cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT) val log = createLog(logDir, logConfig) val errorMsgPrefix = "Compacted topic cannot accept message without key" @@ -2216,7 +2217,7 @@ class UnifiedLogTest { @Test def testAppendMessageWithNullPayload(): Unit = { - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0) val head = LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next() assertEquals(0, head.offset) @@ -2225,7 +2226,7 @@ class UnifiedLogTest { @Test def testAppendWithOutOfOrderOffsetsThrowsException(): Unit = { - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) val appendOffsets = Seq(0L, 1L, 3L, 2L, 4L) val buffer = ByteBuffer.allocate(512) @@ -2246,7 +2247,7 @@ class UnifiedLogTest { @Test def testAppendBelowExpectedOffsetThrowsException(): Unit = { - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) val records = (0 until 2).map(id => new SimpleRecord(id.toString.getBytes)).toArray records.foreach(record => log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, record), leaderEpoch = 0)) @@ -2263,7 +2264,7 @@ class UnifiedLogTest { @Test def testAppendEmptyLogBelowLogStartOffsetThrowsException(): Unit = { createEmptyLogs(logDir, 7) - val log = createLog(logDir, LogConfig(), brokerTopicStats = brokerTopicStats) + val log = createLog(logDir, new LogConfig(new Properties), brokerTopicStats = brokerTopicStats) assertEquals(7L, log.logStartOffset) assertEquals(7L, log.logEndOffset) @@ -2285,7 +2286,7 @@ class UnifiedLogTest { @Test def testAppendWithNoTimestamp(): Unit = { - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(RecordBatch.NO_TIMESTAMP, "key".getBytes, "value".getBytes)), leaderEpoch = 0) } @@ -2294,7 +2295,7 @@ class UnifiedLogTest { def testAppendToOrReadFromLogInFailedLogDir(): Unit = { val pid = 1L val epoch = 0.toShort - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0) assertEquals(0, LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next().offset) val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) @@ -2391,11 +2392,11 @@ class UnifiedLogTest { assertEquals(Some(5), log.latestEpoch) val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, "1000") - logProps.put(LogConfig.IndexIntervalBytesProp, "1") - logProps.put(LogConfig.MaxMessageBytesProp, "65536") - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val downgradedLogConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2") + val downgradedLogConfig = new LogConfig(logProps) log.updateConfig(downgradedLogConfig) LogTestUtils.assertLeaderEpochCacheEmpty(log) @@ -2408,18 +2409,18 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, "1000") - logProps.put(LogConfig.IndexIntervalBytesProp, "1") - logProps.put(LogConfig.MaxMessageBytesProp, "65536") - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val logConfig = LogConfig(logProps) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") + logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2") + val logConfig = new LogConfig(logProps) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), magicValue = RecordVersion.V1.value), leaderEpoch = 5) LogTestUtils.assertLeaderEpochCacheEmpty(log) - logProps.put(LogConfig.MessageFormatVersionProp, "0.11.0") - val upgradedLogConfig = LogConfig(logProps) + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0") + val upgradedLogConfig = new LogConfig(logProps) log.updateConfig(upgradedLogConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) assertEquals(Some(5), log.latestEpoch) @@ -2726,7 +2727,7 @@ class UnifiedLogTest { //Given this partition is on leader epoch 72 val epoch = 72 - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) log.maybeAssignEpochStartOffset(epoch, records.length) //When appending messages as a leader (i.e. assignOffsets = true) @@ -2758,7 +2759,7 @@ class UnifiedLogTest { recs } - val log = createLog(logDir, LogConfig()) + val log = createLog(logDir, new LogConfig(new Properties)) //When appending as follower (assignOffsets = false) for (i <- records.indices) @@ -3549,11 +3550,11 @@ class UnifiedLogTest { log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) assertTrue(log.remoteLogEnabled()) - logConfig = LogTestUtils.createLogConfig(cleanupPolicy = LogConfig.Compact, remoteLogStorageEnable = true) + logConfig = LogTestUtils.createLogConfig(cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT, remoteLogStorageEnable = true) log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) assertFalse(log.remoteLogEnabled()) - logConfig = LogTestUtils.createLogConfig(cleanupPolicy = LogConfig.Compact + "," + LogConfig.Delete, + logConfig = LogTestUtils.createLogConfig(cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE, remoteLogStorageEnable = true) log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) assertFalse(log.remoteLogEnabled()) diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index 29de3c0f24288..3131eb57985d9 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -19,7 +19,6 @@ package kafka.metrics import java.lang.management.ManagementFactory import java.util.Properties - import javax.management.ObjectName import com.yammer.metrics.core.MetricPredicate import org.junit.jupiter.api.Assertions._ @@ -29,8 +28,8 @@ import kafka.utils._ import scala.collection._ import scala.jdk.CollectionConverters._ -import kafka.log.LogConfig import org.apache.kafka.common.TopicPartition +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.server.metrics.KafkaYammerMetrics @@ -173,7 +172,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { val bytesOut = s"${BrokerTopicStats.BytesOutPerSec},topic=$topic" val topicConfig = new Properties - topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, "2") + topicConfig.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2") createTopic(topic, 1, numNodes, topicConfig) // Produce a few messages to create the metrics TestUtils.generateAndProduceMessages(brokers, topic, nMessages) diff --git a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala index c0ed4cdcb89f9..f17b27e1ee54a 100644 --- a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala @@ -16,18 +16,18 @@ */ package kafka.server -import kafka.log.LogConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.Record import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer import org.junit.jupiter.api.AfterEach + import java.util import java.util.{Optional, Properties} - import scala.collection.Seq import scala.jdk.CollectionConverters._ @@ -85,7 +85,7 @@ class BaseFetchRequestTest extends BaseRequestTest { protected def createTopics(numTopics: Int, numPartitions: Int, configs: Map[String, String] = Map.empty): Map[TopicPartition, Int] = { val topics = (0 until numTopics).map(t => s"topic$t") val topicConfig = new Properties - topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString) + topicConfig.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, 2.toString) configs.foreach { case (k, v) => topicConfig.setProperty(k, v) } topics.flatMap { topic => val partitionToLeader = createTopic(topic, numPartitions = numPartitions, replicationFactor = 2, diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala index fc46640c1b65f..cdccee070082e 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala @@ -19,8 +19,8 @@ package kafka.server import java.util import java.util.Properties -import kafka.log.LogConfig import kafka.utils.TestInfoUtils +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 @@ -57,7 +57,7 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic3", numPartitions = 11, replicationFactor = 2, - config = Map(LogConfig.RetentionMsProp -> 4999.toString))), + config = Map(TopicConfig.RETENTION_MS_CONFIG -> 4999.toString))), validateOnly = true)) validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic4", @@ -81,19 +81,19 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest validateErrorCreateTopicsRequests(topicsReq(Seq(topicReq("policy-topic3", numPartitions = 11, replicationFactor = 2, - config = Map(LogConfig.RetentionMsProp -> 5001.toString))), validateOnly = true), + config = Map(TopicConfig.RETENTION_MS_CONFIG -> 5001.toString))), validateOnly = true), Map("policy-topic3" -> error(Errors.POLICY_VIOLATION, Some("RetentionMs should be less than 5000ms if replicationFactor > 5")))) validateErrorCreateTopicsRequests(topicsReq(Seq(topicReq("policy-topic4", numPartitions = 11, replicationFactor = 3, - config = Map(LogConfig.RetentionMsProp -> 5001.toString))), validateOnly = true), + config = Map(TopicConfig.RETENTION_MS_CONFIG -> 5001.toString))), validateOnly = true), Map("policy-topic4" -> error(Errors.POLICY_VIOLATION, Some("RetentionMs should be less than 5000ms if replicationFactor > 5")))) validateErrorCreateTopicsRequests(topicsReq(Seq(topicReq("policy-topic5", assignment = Map(0 -> List(1), 1 -> List(0)), - config = Map(LogConfig.RetentionMsProp -> 5001.toString))), validateOnly = true), + config = Map(TopicConfig.RETENTION_MS_CONFIG -> 5001.toString))), validateOnly = true), Map("policy-topic5" -> error(Errors.POLICY_VIOLATION, Some("Topic partitions should have at least 2 partitions, received 1 for partition 0")))) @@ -165,7 +165,7 @@ object CreateTopicsRequestWithPolicyTest { throw new PolicyViolationException(s"Topics should have at least 5 partitions, received $numPartitions") if (numPartitions > 10) { - if (requestMetadata.configs.asScala.get(LogConfig.RetentionMsProp).fold(true)(_.toInt > 5000)) + if (requestMetadata.configs.asScala.get(TopicConfig.RETENTION_MS_CONFIG).fold(true)(_.toInt > 5000)) throw new PolicyViolationException("RetentionMs should be less than 5000ms if replicationFactor > 5") } else require(requestMetadata.configs.isEmpty, s"Topic configs should be empty, but it is ${requestMetadata.configs}") diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 578e8afb40263..afe9d11e78f6d 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -18,11 +18,11 @@ package kafka.server import java.{lang, util} -import java.util.{Map => JMap, Properties} +import java.util.{Properties, Map => JMap} import java.util.concurrent.CompletionStage import java.util.concurrent.atomic.AtomicReference import kafka.controller.KafkaController -import kafka.log.{LogConfig, LogManager} +import kafka.log.LogManager import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.utils.{KafkaScheduler, TestUtils} import kafka.zk.KafkaZkClient @@ -33,6 +33,7 @@ import org.apache.kafka.common.config.{ConfigException, SslConfigs} import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.server.authorizer._ +import org.apache.kafka.server.log.internals.LogConfig import org.apache.kafka.test.MockMetricsReporter import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -106,7 +107,7 @@ class DynamicBrokerConfigTest { Mockito.when(serverMock.logManager).thenReturn(logManagerMock) Mockito.when(logManagerMock.allLogs).thenReturn(Iterable.empty) - val currentDefaultLogConfig = new AtomicReference(LogConfig()) + val currentDefaultLogConfig = new AtomicReference(new LogConfig(new Properties)) Mockito.when(logManagerMock.currentDefaultConfig).thenAnswer(_ => currentDefaultLogConfig.get()) Mockito.when(logManagerMock.reconfigureDefaultLogConfig(ArgumentMatchers.any(classOf[LogConfig]))) .thenAnswer(invocation => currentDefaultLogConfig.set(invocation.getArgument(0))) @@ -512,7 +513,7 @@ class DynamicBrokerConfigTest { config.dynamicConfig.initialize(None) assertEquals(Defaults.MaxConnections, config.maxConnections) - assertEquals(Defaults.MessageMaxBytes, config.messageMaxBytes) + assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES, config.messageMaxBytes) var newProps = new Properties() newProps.put(KafkaConfig.MaxConnectionsProp, "9999") diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index bd1b506e27c17..80b4327c44e69 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -23,7 +23,6 @@ import java.util.Collections.{singletonList, singletonMap} import java.util.{Collections, Properties} import java.util.concurrent.ExecutionException import kafka.integration.KafkaServerTestHarness -import kafka.log.LogConfig._ import kafka.utils._ import kafka.server.Constants._ import kafka.zk.ConfigEntityChangeNotificationZNode @@ -31,7 +30,7 @@ import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, Config, ConfigEntry} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.config.internals.QuotaConfigs import org.apache.kafka.common.errors.{InvalidRequestException, UnknownTopicOrPartitionException} import org.apache.kafka.common.metrics.Quota @@ -41,6 +40,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.log.internals.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest @@ -67,7 +67,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val newVal: java.lang.Long = 200000L val tp = new TopicPartition("test", 0) val logProps = new Properties() - logProps.put(FlushMessagesProp, oldVal.toString) + logProps.put(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, oldVal.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) @@ -78,7 +78,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(FlushMessagesProp, newVal.toString()), + 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()), @@ -92,7 +92,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } else { val newProps = new Properties() - newProps.setProperty(FlushMessagesProp, newVal.toString()) + newProps.setProperty(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, newVal.toString()) adminZkClient.changeTopicConfig(tp.topic, newProps) } TestUtils.retry(10000) { @@ -106,7 +106,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val tp = new TopicPartition("test", 0) val oldSegmentSize = 1000 val logProps = new Properties() - logProps.put(SegmentBytesProp, oldSegmentSize.toString) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) @@ -119,7 +119,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(SegmentBytesProp, newSegmentSize.toString()), + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString()), SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { @@ -127,7 +127,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } else { val newProps = new Properties() - newProps.put(SegmentBytesProp, newSegmentSize.toString) + newProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString) adminZkClient.changeTopicConfig(tp.topic, newProps) } @@ -147,7 +147,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { def testMessageFormatVersionChange(quorum: String): Unit = { val tp = new TopicPartition("test", 0) val logProps = new Properties() - logProps.put(MessageFormatVersionProp, "0.10.2") + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.10.2") createTopic(tp.topic, 1, 1, logProps) val server = servers.head TestUtils.waitUntilTrue(() => server.logManager.getLog(tp).isDefined, @@ -158,9 +158,9 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(RecordVersion.V2, log.config.recordVersion) val compressionType = CompressionType.LZ4.name - logProps.put(MessageFormatVersionProp, "0.11.0") + logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0") // set compression type so that we can detect when the config change has propagated - logProps.put(CompressionTypeProp, compressionType) + logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType) adminZkClient.changeTopicConfig(tp.topic, logProps) TestUtils.waitUntilTrue(() => server.logManager.getLog(tp).get.config.compressionType == compressionType, @@ -381,7 +381,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { def testConfigChangeOnNonExistingTopic(quorum: String): Unit = { val topic = TestUtils.tempTopic() val logProps = new Properties() - logProps.put(FlushMessagesProp, 10000: java.lang.Integer) + logProps.put(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, 10000: java.lang.Integer) assertThrows(classOf[UnknownTopicOrPartitionException], () => adminZkClient.changeTopicConfig(topic, logProps)) } @@ -392,7 +392,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, topic) - val op = new AlterConfigOp(new ConfigEntry(FlushMessagesProp, "10000"), SET) + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, "10000"), SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get fail("Should fail with UnknownTopicOrPartitionException for topic doesn't exist") } catch { @@ -442,7 +442,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, "") - val op = new AlterConfigOp(new ConfigEntry(FlushMessagesProp, "200000"), SET) + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, "200000"), SET) val future = admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all TestUtils.assertFutureExceptionTypeEquals(future, classOf[InvalidRequestException]) } finally { @@ -457,7 +457,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, "") - val config = new Config(Collections.singleton(new ConfigEntry(FlushMessagesProp, "200000"))) + val config = new Config(Collections.singleton(new ConfigEntry(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, "200000"))) val future = admin.alterConfigs(Map(resource -> config).asJava).all TestUtils.assertFutureExceptionTypeEquals(future, classOf[InvalidRequestException]) } finally { @@ -488,11 +488,11 @@ class DynamicConfigChangeUnitTest { val props: Properties = new Properties() //Given - props.put(LeaderReplicationThrottledReplicasProp, "0:101,0:102,1:101,1:102") + props.put(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:101,0:102,1:101,1:102") //When/Then - assertEquals(Seq(0,1), configHandler.parseThrottledPartitions(props, 102, LeaderReplicationThrottledReplicasProp)) - assertEquals(Seq(), configHandler.parseThrottledPartitions(props, 103, LeaderReplicationThrottledReplicasProp)) + assertEquals(Seq(0,1), configHandler.parseThrottledPartitions(props, 102, LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG)) + assertEquals(Seq(), configHandler.parseThrottledPartitions(props, 103, LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG)) } @Test @@ -501,10 +501,10 @@ class DynamicConfigChangeUnitTest { val props: Properties = new Properties() //Given - props.put(LeaderReplicationThrottledReplicasProp, "*") + props.put(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "*") //When - val result = configHandler.parseThrottledPartitions(props, 102, LeaderReplicationThrottledReplicasProp) + val result = configHandler.parseThrottledPartitions(props, 102, LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG) //Then assertEquals(AllReplicas, result) @@ -514,8 +514,8 @@ class DynamicConfigChangeUnitTest { def shouldParseRegardlessOfWhitespaceAroundValues(): Unit = { def parse(configHandler: TopicConfigHandler, value: String): Seq[Int] = { configHandler.parseThrottledPartitions( - CoreUtils.propsWith(LeaderReplicationThrottledReplicasProp, value), - 102, LeaderReplicationThrottledReplicasProp) + CoreUtils.propsWith(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, value), + 102, LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG) } val configHandler: TopicConfigHandler = new TopicConfigHandler(null, null, null, null) assertEquals(AllReplicas, parse(configHandler, "* ")) @@ -531,10 +531,10 @@ class DynamicConfigChangeUnitTest { val props: Properties = new Properties() //Given - props.put(FollowerReplicationThrottledReplicasProp, "") + props.put(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "") //When - val result = configHandler.parseThrottledPartitions(props, 102, FollowerReplicationThrottledReplicasProp) + val result = configHandler.parseThrottledPartitions(props, 102, LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG) //Then assertEquals(Seq(), result) diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala index 56a3485da4072..aa8a652741ff1 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala @@ -18,10 +18,9 @@ package kafka.server import java.util import java.util.{Optional, Properties} - -import kafka.log.LogConfig import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -65,7 +64,7 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest { configs: Map[String, String] = Map.empty, topicSuffixStart: Int = 0): Map[TopicPartition, Int] = { val topics = (0 until numTopics).map(t => s"topic${t + topicSuffixStart}") val topicConfig = new Properties - topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 1.toString) + topicConfig.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, 1.toString) configs.foreach { case (k, v) => topicConfig.setProperty(k, v) } topics.flatMap { topic => val partitionToLeader = createTopic( @@ -179,7 +178,7 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest { ) val topicConfig = new Properties - topicConfig.put(LogConfig.MessageDownConversionEnableProp, "true") + topicConfig.put(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "true") val topicWithDownConversionEnabledId = TestUtils.createTopicWithAdminRaw( admin, topicWithDownConversionEnabled, diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala index d7ec4c3e365dd..f5425894bc721 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala @@ -17,9 +17,9 @@ package kafka.server -import kafka.log.LogConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} @@ -77,7 +77,7 @@ class FetchRequestMaxBytesTest extends BaseRequestTest { private def createTopics(): Unit = { val topicConfig = new Properties - topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 1.toString) + topicConfig.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, 1.toString) createTopic(testTopic, numPartitions = 1, replicationFactor = 1, diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 98e56e43a1ab5..9104d2840de96 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -16,9 +16,9 @@ */ package kafka.server -import kafka.log.LogConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, RecordBatch} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata} @@ -609,7 +609,7 @@ class FetchRequestTest extends BaseFetchRequestTest { @Test def testZStdCompressedTopic(): Unit = { // ZSTD compressed topic - val topicConfig = Map(LogConfig.CompressionTypeProp -> BrokerCompressionType.ZSTD.name) + val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.ZSTD.name) val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head val topicIds = getTopicIds().asJava val topicNames = topicIds.asScala.map(_.swap).asJava @@ -656,7 +656,7 @@ class FetchRequestTest extends BaseFetchRequestTest { @Test def testZStdCompressedRecords(): Unit = { // Producer compressed topic - val topicConfig = Map(LogConfig.CompressionTypeProp -> BrokerCompressionType.PRODUCER.name) + val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.PRODUCER.name) val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head val topicIds = getTopicIds().asJava val topicNames = topicIds.asScala.map(_.swap).asJava diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestWithLegacyMessageFormatTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestWithLegacyMessageFormatTest.scala index 82e0449c87b48..82ea80e087ea8 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestWithLegacyMessageFormatTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestWithLegacyMessageFormatTest.scala @@ -16,14 +16,14 @@ */ package kafka.server -import kafka.log.LogConfig import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Test -import java.util.Properties +import java.util.Properties import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_2_IV0 import scala.annotation.nowarn @@ -49,7 +49,7 @@ class FetchRequestWithLegacyMessageFormatTest extends BaseFetchRequestTest { val maxPartitionBytes = 200 // Fetch v2 down-converts if the message format is >= 0.11 and we want to avoid // that as it affects the size of the returned buffer - val topicConfig = Map(LogConfig.MessageFormatVersionProp -> IBP_0_10_2_IV0.version) + val topicConfig = Map(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG -> IBP_0_10_2_IV0.version) val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, topicConfig).head val topicIds = getTopicIds().asJava val topicNames = topicIds.asScala.map(_.swap).asJava diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 283a8daec4f64..b77175b0dd153 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -16,21 +16,18 @@ */ package kafka.server -import kafka.log._ import java.io.File - import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api._ import org.junit.jupiter.api.Assertions._ import kafka.utils.{KafkaScheduler, MockTime, TestUtils} - import kafka.cluster.Partition import kafka.server.metadata.MockConfigRepository import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.record.SimpleRecord -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{CleanerConfig, LogDirFailureChannel} class HighwatermarkPersistenceTest { @@ -40,7 +37,7 @@ class HighwatermarkPersistenceTest { val logManagers = configs map { config => TestUtils.createLogManager( logDirs = config.logDirs.map(new File(_)), - cleanerConfig = CleanerConfig()) + cleanerConfig = new CleanerConfig(true)) } val logDirFailureChannels = configs map { config => diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 265a533fbbc35..0137ed42eafd4 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -18,7 +18,6 @@ package kafka.server import kafka.cluster.EndPoint -import kafka.log.LogConfig import kafka.utils.TestUtils.assertBadConfigContainingMessage import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.config.{ConfigException, TopicConfig} @@ -37,6 +36,8 @@ import java.util.{Collections, Properties} import org.apache.kafka.common.Node 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.ServerTopicConfigSynonyms +import org.apache.kafka.server.log.internals.LogConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.junit.jupiter.api.function.Executable @@ -1005,59 +1006,59 @@ class KafkaConfigTest { // Every log config prop must be explicitly accounted for here. // A value other than the default value for this config should be set to ensure that we can check whether // the value is dynamically updatable. - LogConfig.TopicConfigSynonyms.foreach { case (logConfig, kafkaConfigProp) => + ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.forEach { case (logConfig, kafkaConfigProp) => logConfig match { - case LogConfig.CleanupPolicyProp => - assertDynamic(kafkaConfigProp, Defaults.Compact, () => config.logCleanupPolicy) - case LogConfig.CompressionTypeProp => + case TopicConfig.CLEANUP_POLICY_CONFIG => + assertDynamic(kafkaConfigProp, TopicConfig.CLEANUP_POLICY_COMPACT, () => config.logCleanupPolicy) + case TopicConfig.COMPRESSION_TYPE_CONFIG => assertDynamic(kafkaConfigProp, "lz4", () => config.compressionType) - case LogConfig.SegmentBytesProp => + case TopicConfig.SEGMENT_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10000, () => config.logSegmentBytes) - case LogConfig.SegmentMsProp => + case TopicConfig.SEGMENT_MS_CONFIG => assertDynamic(kafkaConfigProp, 10001L, () => config.logRollTimeMillis) - case LogConfig.DeleteRetentionMsProp => + case TopicConfig.DELETE_RETENTION_MS_CONFIG => assertDynamic(kafkaConfigProp, 10002L, () => config.logCleanerDeleteRetentionMs) - case LogConfig.FileDeleteDelayMsProp => + case TopicConfig.FILE_DELETE_DELAY_MS_CONFIG => assertDynamic(kafkaConfigProp, 10003L, () => config.logDeleteDelayMs) - case LogConfig.FlushMessagesProp => + case TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG => assertDynamic(kafkaConfigProp, 10004L, () => config.logFlushIntervalMessages) - case LogConfig.FlushMsProp => + case TopicConfig.FLUSH_MS_CONFIG => assertDynamic(kafkaConfigProp, 10005L, () => config.logFlushIntervalMs) - case LogConfig.MaxCompactionLagMsProp => + case TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG => assertDynamic(kafkaConfigProp, 10006L, () => config.logCleanerMaxCompactionLagMs) - case LogConfig.IndexIntervalBytesProp => + case TopicConfig.INDEX_INTERVAL_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10007, () => config.logIndexIntervalBytes) - case LogConfig.MaxMessageBytesProp => + case TopicConfig.MAX_MESSAGE_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10008, () => config.messageMaxBytes) - case LogConfig.MessageDownConversionEnableProp => + case TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG => assertDynamic(kafkaConfigProp, false, () => config.logMessageDownConversionEnable) - case LogConfig.MessageTimestampDifferenceMaxMsProp => + case TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG => assertDynamic(kafkaConfigProp, 10009, () => config.logMessageTimestampDifferenceMaxMs) - case LogConfig.MessageTimestampTypeProp => + case TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG => assertDynamic(kafkaConfigProp, "LogAppendTime", () => config.logMessageTimestampType.name) - case LogConfig.MinCleanableDirtyRatioProp => + case TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG => assertDynamic(kafkaConfigProp, 0.01, () => config.logCleanerMinCleanRatio) - case LogConfig.MinCompactionLagMsProp => + case TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG => assertDynamic(kafkaConfigProp, 10010L, () => config.logCleanerMinCompactionLagMs) - case LogConfig.MinInSyncReplicasProp => + case TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG => assertDynamic(kafkaConfigProp, 4, () => config.minInSyncReplicas) - case LogConfig.PreAllocateEnableProp => + case TopicConfig.PREALLOCATE_CONFIG => assertDynamic(kafkaConfigProp, true, () => config.logPreAllocateEnable) - case LogConfig.RetentionBytesProp => + case TopicConfig.RETENTION_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10011L, () => config.logRetentionBytes) - case LogConfig.RetentionMsProp => + case TopicConfig.RETENTION_MS_CONFIG => assertDynamic(kafkaConfigProp, 10012L, () => config.logRetentionTimeMillis) - case LogConfig.SegmentIndexBytesProp => + case TopicConfig.SEGMENT_INDEX_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10013, () => config.logIndexSizeMaxBytes) - case LogConfig.SegmentJitterMsProp => + case TopicConfig.SEGMENT_JITTER_MS_CONFIG => assertDynamic(kafkaConfigProp, 10014L, () => config.logRollTimeJitterMillis) - case LogConfig.UncleanLeaderElectionEnableProp => + case TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG => assertDynamic(kafkaConfigProp, true, () => config.uncleanLeaderElectionEnable) - case LogConfig.MessageFormatVersionProp => + case TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG => // not dynamically updatable - case LogConfig.FollowerReplicationThrottledReplicasProp => + case LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config - case LogConfig.LeaderReplicationThrottledReplicasProp => + case LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config case prop => fail(prop + " must be explicitly checked for dynamic updatability. Note that LogConfig(s) require that KafkaConfig value lookups are dynamic and not static values.") diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index 36cb6c5f211ec..d02f59c00258e 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -19,9 +19,9 @@ package kafka.server import java.nio.ByteBuffer import java.util.{Collections, Properties} -import kafka.log.LogConfig import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record._ @@ -83,7 +83,7 @@ class ProduceRequestTest extends BaseRequestTest { val topic = "topic" val partition = 0 val topicConfig = new Properties - topicConfig.setProperty(LogConfig.MessageTimestampDifferenceMaxMsProp, "1000") + topicConfig.setProperty(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000") val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig) val leader = partitionToLeader(partition) @@ -207,7 +207,7 @@ class ProduceRequestTest extends BaseRequestTest { // Create a single-partition topic compressed with ZSTD val topicConfig = new Properties - topicConfig.setProperty(LogConfig.CompressionTypeProp, BrokerCompressionType.ZSTD.name) + topicConfig.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.ZSTD.name) val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig) val leader = partitionToLeader(partition) val memoryRecords = MemoryRecords.withRecords(CompressionType.ZSTD, diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index 1b91f61724bcc..f2db6eeee6ee6 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -21,7 +21,6 @@ import java.util import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit} import java.util.{Optional, Properties} import kafka.api.LeaderAndIsr -import kafka.log.LogConfig import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.MockConfigRepository import kafka.utils.TestUtils.waitUntilTrue @@ -39,7 +38,7 @@ import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.PartitionRegistration -import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.Mockito diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index a5e64a60cfefa..9832059c7c00d 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -58,7 +58,7 @@ import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, C import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 -import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel, LogOffsetMetadata} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig, LogDirFailureChannel, LogOffsetMetadata} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -211,7 +211,7 @@ class ReplicaManagerTest { props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) val config = KafkaConfig.fromProps(props) val logProps = new Properties() - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)), LogConfig(logProps)) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(logProps)) val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1)) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) mockGetAliveBrokerFunctions(metadataCache, aliveBrokers) @@ -1958,7 +1958,7 @@ class ReplicaManagerTest { props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) props.asScala ++= extraProps.asScala val config = KafkaConfig.fromProps(props) - val logConfig = LogConfig() + val logConfig = new LogConfig(new Properties) val logDir = new File(new File(config.logDirs.head), s"$topic-$topicPartition") Files.createDirectories(logDir.toPath) val mockScheduler = new MockScheduler(time) @@ -2299,7 +2299,7 @@ class ReplicaManagerTest { propsModifier.apply(props) val config = KafkaConfig.fromProps(props) val logProps = new Properties() - val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)), LogConfig(logProps)) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(logProps)) val aliveBrokers = aliveBrokerIds.map(brokerId => new Node(brokerId, s"host$brokerId", brokerId)) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 96af81d559166..e8f92ce3e3425 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -18,16 +18,14 @@ package kafka.server import java.util.Properties - -import kafka.log.LogConfig._ import kafka.server.KafkaConfig.fromProps import kafka.server.QuotaType._ import kafka.utils.TestUtils._ import kafka.utils.CoreUtils._ import kafka.utils.TestUtils -import kafka.server.QuorumTestHarness import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.server.log.internals.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -109,9 +107,9 @@ class ReplicationQuotasTest extends QuorumTestHarness { //Either throttle the six leaders or the two followers if (leaderThrottle) - adminZkClient.changeTopicConfig(topic, propsWith(LeaderReplicationThrottledReplicasProp, "0:100,1:101,2:102,3:103,4:104,5:105" )) + adminZkClient.changeTopicConfig(topic, propsWith(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:100,1:101,2:102,3:103,4:104,5:105" )) else - adminZkClient.changeTopicConfig(topic, propsWith(FollowerReplicationThrottledReplicasProp, "0:106,1:106,2:106,3:107,4:107,5:107")) + adminZkClient.changeTopicConfig(topic, propsWith(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:106,1:106,2:106,3:107,4:107,5:107")) //Add data equally to each partition producer = createProducer(plaintextBootstrapServers(brokers), acks = 1) @@ -189,7 +187,7 @@ class ReplicationQuotasTest extends QuorumTestHarness { //Set the throttle to only limit leader adminZkClient.changeBrokerConfig(Seq(100), propsWith(DynamicConfig.Broker.LeaderReplicationThrottledRateProp, throttle.toString)) - adminZkClient.changeTopicConfig(topic, propsWith(LeaderReplicationThrottledReplicasProp, "0:100")) + adminZkClient.changeTopicConfig(topic, propsWith(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, "0:100")) //Add data addData(msgCount, msg) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 8679706777dbc..766861d0a3476 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -19,12 +19,11 @@ import java.util import java.util.concurrent.{Executors, Future, TimeUnit} import java.util.{Collections, Optional, Properties} import kafka.api.LeaderAndIsr -import kafka.log.LogConfig import kafka.network.RequestChannel.Session import kafka.security.authorizer.AclAuthorizer import kafka.utils.TestUtils import org.apache.kafka.common.acl._ -import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection @@ -490,7 +489,7 @@ class RequestQuotaTest extends BaseRequestTest { new AlterConfigsRequest.Builder( Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic), new AlterConfigsRequest.Config(Collections.singleton( - new AlterConfigsRequest.ConfigEntry(LogConfig.MaxMessageBytesProp, "1000000") + new AlterConfigsRequest.ConfigEntry(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "1000000") ))), true) case ApiKeys.ALTER_REPLICA_LOG_DIRS => diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 628d1b66169da..ddacb506778aa 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -21,12 +21,13 @@ import java.io.{ByteArrayOutputStream, File, PrintWriter} import java.nio.ByteBuffer import java.util import java.util.Properties -import kafka.log.{Defaults, LogConfig, LogTestUtils, ProducerStateManagerConfig, UnifiedLog} +import kafka.log.{LogTestUtils, ProducerStateManagerConfig, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} import kafka.server.{BrokerTopicStats, FetchLogEnd, KafkaRaftServer} import kafka.tools.DumpLogSegments.TimeIndexDumpErrors import kafka.utils.{MockTime, TestUtils} import org.apache.kafka.common.Uuid +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.metadata.{PartitionChangeRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.protocol.{ByteBufferAccessor, ObjectSerializationCache} @@ -35,7 +36,7 @@ import org.apache.kafka.common.utils.Utils 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.log.internals.{AppendOrigin, LogDirFailureChannel} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogConfig, LogDirFailureChannel} import org.apache.kafka.snapshot.RecordsSnapshotWriter import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -64,10 +65,10 @@ class DumpLogSegmentsTest { @BeforeEach def setUp(): Unit = { val props = new Properties - props.setProperty(LogConfig.IndexIntervalBytesProp, "128") + props.setProperty(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "128") log = UnifiedLog( dir = logDir, - config = LogConfig(props), + config = new LogConfig(props), logStartOffset = 0L, recoveryPoint = 0L, scheduler = time.scheduler, @@ -312,7 +313,7 @@ class DumpLogSegmentsTest { retentionMillis = 60 * 1000, maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES, maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES, - fileDeleteDelayMs = Defaults.FileDeleteDelayMs, + fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, nodeId = 1 ) ) diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index 1f104a39ff10a..d1eda85729d90 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -19,10 +19,10 @@ package kafka.utils import java.util.Properties import java.util.concurrent.atomic._ import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} -import kafka.log.{LocalLog, LogConfig, LogLoader, LogSegments, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} +import kafka.log.{LocalLog, LogLoader, LogSegments, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} import kafka.server.BrokerTopicStats import kafka.utils.TestUtils.retry -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{LogConfig, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, Timeout} @@ -127,7 +127,7 @@ class SchedulerTest { def testUnscheduleProducerTask(): Unit = { val tmpDir = TestUtils.tempDir() val logDir = TestUtils.randomPartitionLogDir(tmpDir) - val logConfig = LogConfig(new Properties()) + val logConfig = new LogConfig(new Properties()) val brokerTopicStats = new BrokerTopicStats val maxTransactionTimeoutMs = 5 * 60 * 1000 val maxProducerIdExpirationMs = kafka.server.Defaults.ProducerIdExpirationMs diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 8718ff685750a..04c6e3cd060ef 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -70,7 +70,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.controller.QuorumController import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.log.internals.LogDirFailureChannel +import org.apache.kafka.server.log.internals.{CleanerConfig, LogConfig, LogDirFailureChannel} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.zookeeper.KeeperException.SessionExpiredException @@ -1351,9 +1351,9 @@ object TestUtils extends Logging { * Create new LogManager instance with default configuration for testing */ def createLogManager(logDirs: Seq[File] = Seq.empty[File], - defaultConfig: LogConfig = LogConfig(), + defaultConfig: LogConfig = new LogConfig(new Properties), configRepository: ConfigRepository = new MockConfigRepository, - cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), + cleanerConfig: CleanerConfig = new CleanerConfig(false), time: MockTime = new MockTime(), interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest, recoveryThreadsPerDataDir: Int = 4): LogManager = { @@ -2156,8 +2156,8 @@ object TestUtils extends Logging { val throttles = allReplicasByPartition.groupBy(_._1.topic()).map { case (topic, replicasByPartition) => new ConfigResource(TOPIC, topic) -> Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET), - new AlterConfigOp(new ConfigEntry(LogConfig.FollowerReplicationThrottledReplicasProp, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET) + new AlterConfigOp(new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET), + new AlterConfigOp(new ConfigEntry(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET) ).asJavaCollection } adminClient.incrementalAlterConfigs(throttles.asJava).all().get() @@ -2167,8 +2167,8 @@ object TestUtils extends Logging { val throttles = partitions.map { tp => new ConfigResource(TOPIC, tp.topic()) -> Seq( - new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, ""), AlterConfigOp.OpType.DELETE), - new AlterConfigOp(new ConfigEntry(LogConfig.FollowerReplicationThrottledReplicasProp, ""), AlterConfigOp.OpType.DELETE) + new AlterConfigOp(new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""), AlterConfigOp.OpType.DELETE), + new AlterConfigOp(new ConfigEntry(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""), AlterConfigOp.OpType.DELETE) ).asJavaCollection }.toMap adminClient.incrementalAlterConfigs(throttles.asJava).all().get() diff --git a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala index 5bdbc730aa5ed..1e19fe1bcc6d9 100644 --- a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala @@ -18,21 +18,20 @@ package kafka.admin import java.util import java.util.Properties - import kafka.controller.ReplicaAssignment -import kafka.log._ import kafka.server.DynamicConfig.Broker._ import kafka.server.KafkaConfig._ import kafka.server.{ConfigType, KafkaConfig, KafkaServer, QuorumTestHarness} import kafka.utils.CoreUtils._ import kafka.utils.TestUtils._ import kafka.utils.{Logging, TestUtils} -import kafka.zk.{AdminZkClient, KafkaZkClient, ConfigEntityTypeZNode} +import kafka.zk.{AdminZkClient, ConfigEntityTypeZNode, KafkaZkClient} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.internals.QuotaConfigs import org.apache.kafka.common.errors.{InvalidReplicaAssignmentException, InvalidTopicException, TopicExistsException} import org.apache.kafka.common.metrics.Quota +import org.apache.kafka.server.log.internals.LogConfig import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -200,10 +199,10 @@ class AdminZkClientTest extends QuorumTestHarness with Logging with RackAwareTes def makeConfig(messageSize: Int, retentionMs: Long, throttledLeaders: String, throttledFollowers: String) = { val props = new Properties() - props.setProperty(LogConfig.MaxMessageBytesProp, messageSize.toString) - props.setProperty(LogConfig.RetentionMsProp, retentionMs.toString) - props.setProperty(LogConfig.LeaderReplicationThrottledReplicasProp, throttledLeaders) - props.setProperty(LogConfig.FollowerReplicationThrottledReplicasProp, throttledFollowers) + props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, messageSize.toString) + props.setProperty(TopicConfig.RETENTION_MS_CONFIG, retentionMs.toString) + props.setProperty(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, throttledLeaders) + props.setProperty(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, throttledFollowers) props } @@ -222,8 +221,8 @@ class AdminZkClientTest extends QuorumTestHarness with Logging with RackAwareTes assertTrue(log.isDefined) assertEquals(retentionMs, log.get.config.retentionMs) assertEquals(messageSize, log.get.config.maxMessageSize) - checkList(log.get.config.LeaderReplicationThrottledReplicas, throttledLeaders) - checkList(log.get.config.FollowerReplicationThrottledReplicas, throttledFollowers) + checkList(log.get.config.leaderReplicationThrottledReplicas, throttledLeaders) + checkList(log.get.config.followerReplicationThrottledReplicas, throttledFollowers) assertEquals(quotaManagerIsThrottled, server.quotaManagers.leader.isThrottled(tp)) } } @@ -253,15 +252,15 @@ class AdminZkClientTest extends QuorumTestHarness with Logging with RackAwareTes //Now delete the config adminZkClient.changeTopicConfig(topic, new Properties) - checkConfig(Defaults.MaxMessageSize, Defaults.RetentionMs, "", "", quotaManagerIsThrottled = false) + checkConfig(LogConfig.DEFAULT_MAX_MESSAGE_BYTES, LogConfig.DEFAULT_RETENTION_MS, "", "", quotaManagerIsThrottled = false) //Add config back adminZkClient.changeTopicConfig(topic, makeConfig(maxMessageSize, retentionMs, "0:0,1:0,2:0", "0:1,1:1,2:1")) checkConfig(maxMessageSize, retentionMs, "0:0,1:0,2:0", "0:1,1:1,2:1", quotaManagerIsThrottled = true) //Now ensure updating to "" removes the throttled replica list also - adminZkClient.changeTopicConfig(topic, propsWith((LogConfig.FollowerReplicationThrottledReplicasProp, ""), (LogConfig.LeaderReplicationThrottledReplicasProp, ""))) - checkConfig(Defaults.MaxMessageSize, Defaults.RetentionMs, "", "", quotaManagerIsThrottled = false) + adminZkClient.changeTopicConfig(topic, propsWith((LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""), (LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""))) + checkConfig(LogConfig.DEFAULT_MAX_MESSAGE_BYTES, LogConfig.DEFAULT_RETENTION_MS, "", "", quotaManagerIsThrottled = false) } @Test diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 416abd23eb23b..b60004e6cee72 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -22,7 +22,6 @@ import java.util.{Collections, Properties} import kafka.api.LeaderAndIsr import kafka.cluster.{Broker, EndPoint} import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} -import kafka.log.LogConfig import kafka.security.authorizer.AclEntry import kafka.server.{ConfigType, KafkaConfig, QuorumTestHarness} import kafka.utils.CoreUtils @@ -30,6 +29,7 @@ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult import kafka.zookeeper._ import org.apache.kafka.common.acl.AclOperation.READ import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.feature.Features._ import org.apache.kafka.common.feature.{Features, SupportedVersionRange} @@ -44,6 +44,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.LogConfig import org.apache.zookeeper.KeeperException.{Code, NoAuthException, NoNodeException, NodeExistsException} import org.apache.zookeeper.{CreateMode, ZooDefs} import org.apache.zookeeper.client.ZKClientConfig @@ -735,7 +736,7 @@ class KafkaZkClientTest extends QuorumTestHarness { zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic1, logProps) assertEquals(logProps, zkClient.getEntityConfigs(ConfigType.Topic, topic1)) - logProps.remove(LogConfig.CleanupPolicyProp) + logProps.remove(TopicConfig.CLEANUP_POLICY_CONFIG) zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic1, logProps) assertEquals(logProps, zkClient.getEntityConfigs(ConfigType.Topic, topic1)) @@ -765,9 +766,9 @@ class KafkaZkClientTest extends QuorumTestHarness { private def createLogProps(bytesProp: Int): Properties = { val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, bytesProp.toString) - logProps.put(LogConfig.SegmentIndexBytesProp, bytesProp.toString) - logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, bytesProp.toString) + logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, bytesProp.toString) + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) logProps } @@ -775,7 +776,7 @@ class KafkaZkClientTest extends QuorumTestHarness { @Test def testGetLogConfigs(): Unit = { - val emptyConfig = LogConfig(Collections.emptyMap()) + val emptyConfig = new LogConfig(Collections.emptyMap()) assertEquals((Map(topic1 -> emptyConfig), Map.empty), zkClient.getLogConfigs(Set(topic1), Collections.emptyMap()), "Non existent config, no defaults") @@ -783,22 +784,22 @@ class KafkaZkClientTest extends QuorumTestHarness { val logProps2 = createLogProps(2048) zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic1, logProps) - assertEquals((Map(topic1 -> LogConfig(logProps), topic2 -> emptyConfig), Map.empty), + assertEquals((Map(topic1 -> new LogConfig(logProps), topic2 -> emptyConfig), Map.empty), zkClient.getLogConfigs(Set(topic1, topic2), Collections.emptyMap()), "One existing and one non-existent topic") zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic2, logProps2) - assertEquals((Map(topic1 -> LogConfig(logProps), topic2 -> LogConfig(logProps2)), Map.empty), + assertEquals((Map(topic1 -> new LogConfig(logProps), topic2 -> new LogConfig(logProps2)), Map.empty), zkClient.getLogConfigs(Set(topic1, topic2), Collections.emptyMap()), "Two existing topics") val logProps1WithMoreValues = createLogProps(1024) - logProps1WithMoreValues.put(LogConfig.SegmentJitterMsProp, "100") - logProps1WithMoreValues.put(LogConfig.SegmentBytesProp, "1024") + logProps1WithMoreValues.put(TopicConfig.SEGMENT_JITTER_MS_CONFIG, "100") + logProps1WithMoreValues.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") - assertEquals((Map(topic1 -> LogConfig(logProps1WithMoreValues)), Map.empty), + assertEquals((Map(topic1 -> new LogConfig(logProps1WithMoreValues)), Map.empty), zkClient.getLogConfigs(Set(topic1), - Map[String, AnyRef](LogConfig.SegmentJitterMsProp -> "100", LogConfig.SegmentBytesProp -> "128").asJava), + Map[String, AnyRef](TopicConfig.SEGMENT_JITTER_MS_CONFIG -> "100", TopicConfig.SEGMENT_BYTES_CONFIG -> "128").asJava), "Config with defaults") } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java index 107b0e16d7f60..0c3871af43732 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java @@ -21,10 +21,7 @@ import kafka.cluster.DelayedOperations; import kafka.cluster.AlterPartitionListener; import kafka.cluster.Partition; -import kafka.log.CleanerConfig; -import kafka.log.Defaults; import kafka.log.LogAppendInfo; -import kafka.log.LogConfig; import kafka.log.LogManager; import kafka.server.AlterPartitionManager; import kafka.server.BrokerFeatures; @@ -72,6 +69,8 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.CleanerConfig; +import org.apache.kafka.server.log.internals.LogConfig; import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; @@ -110,7 +109,6 @@ @Measurement(iterations = 15) @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.NANOSECONDS) - public class ReplicaFetcherThreadBenchmark { @Param({"100", "500", "1000", "5000"}) private int partitionCount; @@ -144,7 +142,7 @@ public void setup() throws IOException { setInitialOfflineDirs(Collections.emptyList()). setConfigRepository(new MockConfigRepository()). setInitialDefaultConfig(logConfig). - setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5")). + setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false)). setRecoveryThreadsPerDataDir(1). setFlushCheckMs(1000L). setFlushRecoveryOffsetCheckpointMs(10000L). @@ -288,18 +286,7 @@ public int numDelayedDelete() { } private static LogConfig createLogConfig() { - Properties logProps = new Properties(); - logProps.put(LogConfig.SegmentMsProp(), Defaults.SegmentMs()); - logProps.put(LogConfig.SegmentBytesProp(), Defaults.SegmentSize()); - logProps.put(LogConfig.RetentionMsProp(), Defaults.RetentionMs()); - logProps.put(LogConfig.RetentionBytesProp(), Defaults.RetentionSize()); - logProps.put(LogConfig.SegmentJitterMsProp(), Defaults.SegmentJitterMs()); - logProps.put(LogConfig.CleanupPolicyProp(), Defaults.CleanupPolicy()); - logProps.put(LogConfig.MaxMessageBytesProp(), Defaults.MaxMessageSize()); - logProps.put(LogConfig.IndexIntervalBytesProp(), Defaults.IndexInterval()); - logProps.put(LogConfig.SegmentIndexBytesProp(), Defaults.MaxIndexSize()); - logProps.put(LogConfig.FileDeleteDelayMsProp(), Defaults.FileDeleteDelayMs()); - return LogConfig.apply(logProps, new scala.collection.immutable.HashSet<>()); + return new LogConfig(new Properties()); } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java index 7c5b342d9cb80..131276f98828e 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java @@ -20,9 +20,6 @@ import kafka.cluster.DelayedOperations; import kafka.cluster.AlterPartitionListener; import kafka.cluster.Partition; -import kafka.log.CleanerConfig; -import kafka.log.Defaults; -import kafka.log.LogConfig; import kafka.log.LogManager; import kafka.server.AlterPartitionManager; import kafka.server.BrokerTopicStats; @@ -40,6 +37,8 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.CleanerConfig; +import org.apache.kafka.server.log.internals.LogConfig; import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; @@ -76,7 +75,6 @@ @Measurement(iterations = 15) @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.NANOSECONDS) - public class PartitionMakeFollowerBenchmark { private LogManager logManager; private File logDir = new File(System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString()); @@ -94,7 +92,7 @@ public void setup() throws IOException { throw new IOException("error creating test directory"); scheduler.startup(); - LogConfig logConfig = createLogConfig(); + LogConfig logConfig = new LogConfig(new Properties()); BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); LogDirFailureChannel logDirFailureChannel = Mockito.mock(LogDirFailureChannel.class); @@ -103,7 +101,7 @@ public void setup() throws IOException { setInitialOfflineDirs(Collections.emptyList()). setConfigRepository(new MockConfigRepository()). setInitialDefaultConfig(logConfig). - setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5")). + setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false)). setRecoveryThreadsPerDataDir(1). setFlushCheckMs(1000L). setFlushRecoveryOffsetCheckpointMs(10000L). @@ -162,19 +160,4 @@ public boolean testMakeFollower() { .setIsNew(true); return partition.makeFollower(partitionState, offsetCheckpoints, topicId); } - - private static LogConfig createLogConfig() { - Properties logProps = new Properties(); - logProps.put(LogConfig.SegmentMsProp(), Defaults.SegmentMs()); - logProps.put(LogConfig.SegmentBytesProp(), Defaults.SegmentSize()); - logProps.put(LogConfig.RetentionMsProp(), Defaults.RetentionMs()); - logProps.put(LogConfig.RetentionBytesProp(), Defaults.RetentionSize()); - logProps.put(LogConfig.SegmentJitterMsProp(), Defaults.SegmentJitterMs()); - logProps.put(LogConfig.CleanupPolicyProp(), Defaults.CleanupPolicy()); - logProps.put(LogConfig.MaxMessageBytesProp(), Defaults.MaxMessageSize()); - logProps.put(LogConfig.IndexIntervalBytesProp(), Defaults.IndexInterval()); - logProps.put(LogConfig.SegmentIndexBytesProp(), Defaults.MaxIndexSize()); - logProps.put(LogConfig.FileDeleteDelayMsProp(), Defaults.FileDeleteDelayMs()); - return LogConfig.apply(logProps, new scala.collection.immutable.HashSet<>()); - } } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java index f86b942c3c885..26631b36f29ff 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java @@ -21,9 +21,6 @@ import kafka.cluster.AlterPartitionListener; import kafka.cluster.Partition; import kafka.cluster.Replica; -import kafka.log.CleanerConfig; -import kafka.log.Defaults; -import kafka.log.LogConfig; import kafka.log.LogManager; import kafka.server.AlterPartitionManager; import kafka.server.BrokerTopicStats; @@ -37,6 +34,8 @@ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.CleanerConfig; +import org.apache.kafka.server.log.internals.LogConfig; import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.apache.kafka.server.log.internals.LogOffsetMetadata; import org.mockito.Mockito; @@ -92,7 +91,7 @@ public void setUp() { setInitialOfflineDirs(Collections.emptyList()). setConfigRepository(new MockConfigRepository()). setInitialDefaultConfig(logConfig). - setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5")). + setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false)). setRecoveryThreadsPerDataDir(1). setFlushCheckMs(1000L). setFlushRecoveryOffsetCheckpointMs(10000L). @@ -153,18 +152,7 @@ public void tearDown() { } private LogConfig createLogConfig() { - Properties logProps = new Properties(); - logProps.put(LogConfig.SegmentMsProp(), Defaults.SegmentMs()); - logProps.put(LogConfig.SegmentBytesProp(), Defaults.SegmentSize()); - logProps.put(LogConfig.RetentionMsProp(), Defaults.RetentionMs()); - logProps.put(LogConfig.RetentionBytesProp(), Defaults.RetentionSize()); - logProps.put(LogConfig.SegmentJitterMsProp(), Defaults.SegmentJitterMs()); - logProps.put(LogConfig.CleanupPolicyProp(), Defaults.CleanupPolicy()); - logProps.put(LogConfig.MaxMessageBytesProp(), Defaults.MaxMessageSize()); - logProps.put(LogConfig.IndexIntervalBytesProp(), Defaults.IndexInterval()); - logProps.put(LogConfig.SegmentIndexBytesProp(), Defaults.MaxIndexSize()); - logProps.put(LogConfig.FileDeleteDelayMsProp(), Defaults.FileDeleteDelayMs()); - return LogConfig.apply(logProps, new scala.collection.immutable.HashSet<>()); + return new LogConfig(new Properties()); } @Benchmark diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java index bb07e63868fed..7e565c13c8adb 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java @@ -17,13 +17,13 @@ package org.apache.kafka.jmh.server; import kafka.cluster.Partition; -import kafka.log.CleanerConfig; -import kafka.log.LogConfig; import kafka.log.LogManager; import kafka.server.AlterPartitionManager; import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.KafkaConfig; +import org.apache.kafka.server.log.internals.CleanerConfig; +import org.apache.kafka.server.log.internals.LogConfig; import org.apache.kafka.server.log.internals.LogDirFailureChannel; import kafka.server.MetadataCache; import kafka.server.QuotaFactory; @@ -56,6 +56,7 @@ import java.io.File; import java.util.ArrayList; import java.util.List; +import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -106,9 +107,8 @@ public void setup() { final List files = JavaConverters.seqAsJavaList(brokerProperties.logDirs()).stream().map(File::new).collect(Collectors.toList()); this.logManager = TestUtils.createLogManager(JavaConverters.asScalaBuffer(files), - LogConfig.apply(), new MockConfigRepository(), CleanerConfig.apply(1, 4 * 1024 * 1024L, 0.9d, - 1024 * 1024, 32 * 1024 * 1024, - Double.MAX_VALUE, 15 * 1000, true, "MD5"), time, MetadataVersion.latest(), 4); + new LogConfig(new Properties()), new MockConfigRepository(), new CleanerConfig(1, 4 * 1024 * 1024L, 0.9d, + 1024 * 1024, 32 * 1024 * 1024, Double.MAX_VALUE, 15 * 1000, true), time, MetadataVersion.latest(), 4); scheduler.startup(); final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); final MetadataCache metadataCache = diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java index 68880830cae7f..deaca135ee7a9 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java @@ -17,9 +17,6 @@ package org.apache.kafka.jmh.server; import kafka.cluster.Partition; -import kafka.log.CleanerConfig; -import kafka.log.Defaults; -import kafka.log.LogConfig; import kafka.log.LogManager; import kafka.server.AlterPartitionManager; import kafka.server.BrokerFeatures; @@ -44,6 +41,8 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.CleanerConfig; +import org.apache.kafka.server.log.internals.LogConfig; import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -117,10 +116,10 @@ public void setup() { final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); final List files = JavaConverters.seqAsJavaList(brokerProperties.logDirs()).stream().map(File::new).collect(Collectors.toList()); - CleanerConfig cleanerConfig = CleanerConfig.apply(1, + CleanerConfig cleanerConfig = new CleanerConfig(1, 4 * 1024 * 1024L, 0.9d, 1024 * 1024, 32 * 1024 * 1024, - Double.MAX_VALUE, 15 * 1000, true, "MD5"); + Double.MAX_VALUE, 15 * 1000, true); ConfigRepository configRepository = new MockConfigRepository(); this.logManager = new LogManagerBuilder(). @@ -184,18 +183,7 @@ public void tearDown() throws Exception { } private static LogConfig createLogConfig() { - Properties logProps = new Properties(); - logProps.put(LogConfig.SegmentMsProp(), Defaults.SegmentMs()); - logProps.put(LogConfig.SegmentBytesProp(), Defaults.SegmentSize()); - logProps.put(LogConfig.RetentionMsProp(), Defaults.RetentionMs()); - logProps.put(LogConfig.RetentionBytesProp(), Defaults.RetentionSize()); - logProps.put(LogConfig.SegmentJitterMsProp(), Defaults.SegmentJitterMs()); - logProps.put(LogConfig.CleanupPolicyProp(), Defaults.CleanupPolicy()); - logProps.put(LogConfig.MaxMessageBytesProp(), Defaults.MaxMessageSize()); - logProps.put(LogConfig.IndexIntervalBytesProp(), Defaults.IndexInterval()); - logProps.put(LogConfig.SegmentIndexBytesProp(), Defaults.MaxIndexSize()); - logProps.put(LogConfig.FileDeleteDelayMsProp(), Defaults.FileDeleteDelayMs()); - return LogConfig.apply(logProps, new scala.collection.immutable.HashSet<>()); + return new LogConfig(new Properties()); } @Benchmark diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java index d15d6623e38b8..8b770ff66c5bb 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import java.util.function.Function; +import org.apache.kafka.server.config.ConfigSynonym; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index bbca7bfbf6b67..b4ba2e5bd6a7c 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -24,7 +24,7 @@ import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; -import org.apache.kafka.metadata.ConfigSynonym; +import org.apache.kafka.server.config.ConfigSynonym; import org.apache.kafka.metadata.KafkaConfigSchema; import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -54,7 +54,7 @@ import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; import static org.apache.kafka.common.metadata.MetadataRecordType.CONFIG_RECORD; -import static org.apache.kafka.metadata.ConfigSynonym.HOURS_TO_MILLISECONDS; +import static org.apache.kafka.server.config.ConfigSynonym.HOURS_TO_MILLISECONDS; import static org.junit.jupiter.api.Assertions.assertEquals; diff --git a/metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java b/metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java index 93f63c21091f3..deff4f801b2fc 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/ConfigSynonymTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.metadata; +import org.apache.kafka.server.config.ConfigSynonym; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; diff --git a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java index 36089d0f9a4f2..2f870057e1db6 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.requests.DescribeConfigsResponse; +import org.apache.kafka.server.config.ConfigSynonym; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -32,7 +33,7 @@ import static java.util.Collections.emptyList; import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; -import static org.apache.kafka.metadata.ConfigSynonym.HOURS_TO_MILLISECONDS; +import static org.apache.kafka.server.config.ConfigSynonym.HOURS_TO_MILLISECONDS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; diff --git a/metadata/src/main/java/org/apache/kafka/metadata/ConfigSynonym.java b/server-common/src/main/java/org/apache/kafka/server/config/ConfigSynonym.java similarity index 98% rename from metadata/src/main/java/org/apache/kafka/metadata/ConfigSynonym.java rename to server-common/src/main/java/org/apache/kafka/server/config/ConfigSynonym.java index d331a476c92c0..b77efe0b408e9 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/ConfigSynonym.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ConfigSynonym.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.metadata; +package org.apache.kafka.server.config; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 new file mode 100644 index 0000000000000..c6a8c61390d94 --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -0,0 +1,140 @@ +/* + * 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 static java.util.Arrays.asList; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.utils.Utils; + +public final class ServerTopicConfigSynonyms { + private static final String LOG_PREFIX = "log."; + private static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; + + /** + * Maps topic configurations to their equivalent broker configurations. + * + * Topics can be configured either by setting their dynamic topic configurations, or by + * setting equivalent broker configurations. For historical reasons, the equivalent broker + * configurations have different names. This table maps each topic configuration to its + * equivalent broker configurations. + * + * In some cases, the equivalent broker configurations must be transformed before they + * can be used. For example, log.roll.hours must be converted to milliseconds before it + * can be used as the value of segment.ms. + * + * The broker configurations will be used in the order specified here. In other words, if + * both the first and the second synonyms are configured, we will use only the value of + * the first synonym and ignore the second. + */ + // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` + @SuppressWarnings("deprecation") + public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Collections.unmodifiableMap(Utils.mkMap( + sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), + listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, + new ConfigSynonym("roll.ms"), + new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), + listWithLogPrefix(TopicConfig.SEGMENT_JITTER_MS_CONFIG, + new ConfigSynonym("roll.jitter.ms"), + new ConfigSynonym("roll.jitter.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), + singleWithLogPrefix(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, "index.size.max.bytes"), + singleWithLogPrefix(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, "flush.interval.messages"), + listWithLogPrefix(TopicConfig.FLUSH_MS_CONFIG, + new ConfigSynonym("flush.interval.ms"), + new ConfigSynonym("flush.scheduler.interval.ms")), + sameNameWithLogPrefix(TopicConfig.RETENTION_BYTES_CONFIG), + listWithLogPrefix(TopicConfig.RETENTION_MS_CONFIG, + new ConfigSynonym("retention.ms"), + new ConfigSynonym("retention.minutes", ConfigSynonym.MINUTES_TO_MILLISECONDS), + new ConfigSynonym("retention.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), + single(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "message.max.bytes"), + sameNameWithLogPrefix(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG), + sameNameWithLogCleanerPrefix(TopicConfig.DELETE_RETENTION_MS_CONFIG), + sameNameWithLogCleanerPrefix(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG), + sameNameWithLogCleanerPrefix(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG), + singleWithLogPrefix(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "segment.delete.delay.ms"), + singleWithLogCleanerPrefix(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "min.cleanable.ratio"), + sameNameWithLogPrefix(TopicConfig.CLEANUP_POLICY_CONFIG), + sameName(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG), + sameName(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG), + sameName(TopicConfig.COMPRESSION_TYPE_CONFIG), + sameNameWithLogPrefix(TopicConfig.PREALLOCATE_CONFIG), + sameNameWithLogPrefix(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG), + sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG), + sameNameWithLogPrefix(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG), + sameNameWithLogPrefix(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG) + )); + + /** + * Map topic config to the server config with the highest priority. Some of these have additional + * synonyms that can be obtained using [[kafka.server.DynamicBrokerConfig#brokerConfigSynonyms]] + * or using [[AllTopicConfigSynonyms]] + */ + public static final Map TOPIC_CONFIG_SYNONYMS = Collections.unmodifiableMap( + ALL_TOPIC_CONFIG_SYNONYMS.entrySet() + .stream() + .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().get(0).name()))); + + /** + * Return the server config with the highest priority for `topicConfigName` if it exists. Otherwise, + * throw NoSuchElementException. + */ + public static String serverSynonym(String topicConfigName) { + String serverSynonym = TOPIC_CONFIG_SYNONYMS.get(topicConfigName); + if (serverSynonym == null) + throw new NoSuchElementException("No server synonym found for " + topicConfigName); + return serverSynonym; + } + + private static Entry> sameName(String configName) { + return Utils.mkEntry(configName, asList(new ConfigSynonym(configName))); + } + + private static Entry> sameNameWithLogPrefix(String configName) { + return Utils.mkEntry(configName, asList(new ConfigSynonym(LOG_PREFIX + configName))); + } + + private static Entry> sameNameWithLogCleanerPrefix(String configName) { + return Utils.mkEntry(configName, asList(new ConfigSynonym(LOG_CLEANER_PREFIX + configName))); + } + + private static Entry> singleWithLogPrefix(String topicConfigName, String brokerConfigName) { + return Utils.mkEntry(topicConfigName, asList(new ConfigSynonym(LOG_PREFIX + brokerConfigName))); + } + + private static Entry> singleWithLogCleanerPrefix(String topicConfigName, String brokerConfigName) { + return Utils.mkEntry(topicConfigName, asList(new ConfigSynonym(LOG_CLEANER_PREFIX + brokerConfigName))); + } + + private static Entry> listWithLogPrefix(String topicConfigName, ConfigSynonym... synonyms) { + List synonymsWithPrefix = Arrays.stream(synonyms) + .map(s -> new ConfigSynonym(LOG_PREFIX + s.name(), s.converter())) + .collect(Collectors.toList()); + return Utils.mkEntry(topicConfigName, synonymsWithPrefix); + } + + private static Entry> single(String topicConfigName, String brokerConfigName) { + return Utils.mkEntry(topicConfigName, asList(new ConfigSynonym(brokerConfigName))); + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/CleanerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/internals/CleanerConfig.java new file mode 100644 index 0000000000000..3029d4c09912d --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/CleanerConfig.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.internals; + +/** + * Configuration parameters for the log cleaner. + */ +public class CleanerConfig { + public static final String HASH_ALGORITHM = "MD5"; + + public final int numThreads; + public final long dedupeBufferSize; + public final double dedupeBufferLoadFactor; + public final int ioBufferSize; + public final int maxMessageSize; + public final double maxIoBytesPerSecond; + public final long backoffMs; + public final boolean enableCleaner; + + public CleanerConfig(boolean enableCleaner) { + this(1, 4 * 1024 * 1024, 0.9, 1024 * 1024, + 32 * 1024 * 1024, Double.MAX_VALUE, 15 * 1000, enableCleaner); + } + + /** + * Create an instance of this class. + * + * @param numThreads The number of cleaner threads to run + * @param dedupeBufferSize The total memory used for log deduplication + * @param dedupeBufferLoadFactor The maximum percent full for the deduplication buffer + * @param maxMessageSize The maximum size of a message that can appear in the log + * @param maxIoBytesPerSecond The maximum read and write I/O that all cleaner threads are allowed to do + * @param backoffMs The amount of time to wait before rechecking if no logs are eligible for cleaning + * @param enableCleaner Allows completely disabling the log cleaner + */ + public CleanerConfig(int numThreads, + long dedupeBufferSize, + double dedupeBufferLoadFactor, + int ioBufferSize, + int maxMessageSize, + double maxIoBytesPerSecond, + long backoffMs, + boolean enableCleaner) { + this.numThreads = numThreads; + this.dedupeBufferSize = dedupeBufferSize; + this.dedupeBufferLoadFactor = dedupeBufferLoadFactor; + this.ioBufferSize = ioBufferSize; + this.maxMessageSize = maxMessageSize; + this.maxIoBytesPerSecond = maxIoBytesPerSecond; + this.backoffMs = backoffMs; + this.enableCleaner = enableCleaner; + } + + public String hashAlgorithm() { + return HASH_ALGORITHM; + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/LogConfig.java b/storage/src/main/java/org/apache/kafka/server/log/internals/LogConfig.java new file mode 100644 index 0000000000000..25c29dc822b1b --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/LogConfig.java @@ -0,0 +1,504 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.internals; + +import static java.util.Arrays.asList; +import static org.apache.kafka.common.config.ConfigDef.Range.between; +import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; +import static org.apache.kafka.common.config.ConfigDef.Type.DOUBLE; +import static org.apache.kafka.common.config.ConfigDef.Type.LIST; +import static org.apache.kafka.common.config.ConfigDef.Type.LONG; +import static org.apache.kafka.common.config.ConfigDef.Type.STRING; +import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1; +import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; +import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM; +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; +import static org.apache.kafka.common.config.ConfigDef.Type.INT; +import static org.apache.kafka.common.config.ConfigDef.ValidString.in; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.ConfigKey; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.ValidList; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.InvalidConfigurationException; +import org.apache.kafka.common.record.LegacyRecord; +import org.apache.kafka.common.record.RecordVersion; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.utils.ConfigUtils; +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.ServerTopicConfigSynonyms; +import org.apache.kafka.server.record.BrokerCompressionType; + +public class LogConfig extends AbstractConfig { + + public static class MessageFormatVersion { + private final String messageFormatVersionString; + private final String interBrokerProtocolVersionString; + private final MetadataVersion messageFormatVersion; + private final MetadataVersion interBrokerProtocolVersion; + + public MessageFormatVersion(String messageFormatVersionString, String interBrokerProtocolVersionString) { + this.messageFormatVersionString = messageFormatVersionString; + this.interBrokerProtocolVersionString = interBrokerProtocolVersionString; + this.messageFormatVersion = MetadataVersion.fromVersionString(messageFormatVersionString); + this.interBrokerProtocolVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString); + } + + public MetadataVersion messageFormatVersion() { + return messageFormatVersion; + } + + public MetadataVersion interBrokerProtocolVersion() { + return interBrokerProtocolVersion; + } + + public boolean shouldIgnore() { + return shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion); + } + + public boolean shouldWarn() { + return interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1) + && messageFormatVersion.highestSupportedRecordVersion().precedes(RecordVersion.V2); + } + + @SuppressWarnings("deprecation") + public String topicWarningMessage(String topicName) { + return "Topic configuration " + TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG + " with value `" + + messageFormatVersionString + "` is ignored for `" + topicName + "` 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."; + } + } + + public static class RemoteLogConfig { + + public final boolean remoteStorageEnable; + + public final long localRetentionMs; + public final long localRetentionBytes; + + private RemoteLogConfig(LogConfig config, long retentionMs, long retentionSize) { + this.remoteStorageEnable = config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); + + long localLogRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); + + // -2 indicates to derive value from retentionMs property. + if (localLogRetentionMs == -2) + this.localRetentionMs = retentionMs; + else { + // Added validation here to check the effective value should not be more than RetentionMs. + if (localLogRetentionMs == -1 && retentionMs != -1) + throw new ConfigException(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localLogRetentionMs, + "Value must not be -1 as " + TopicConfig.RETENTION_MS_CONFIG + " value is set as " + retentionMs); + + if (localLogRetentionMs > retentionMs) + throw new ConfigException(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localLogRetentionMs, + "Value must not be more than property: " + TopicConfig.RETENTION_MS_CONFIG + " value."); + + this.localRetentionMs = localLogRetentionMs; + } + + long localLogRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); + + // -2 indicates to derive value from retentionSize property. + if (localLogRetentionBytes == -2) + this.localRetentionBytes = retentionSize; + else { + // Added validation here to check the effective value should not be more than RetentionBytes. + if (localLogRetentionBytes == -1 && retentionSize != -1) + throw new ConfigException(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localLogRetentionBytes, + "Value must not be -1 as " + TopicConfig.RETENTION_BYTES_CONFIG + " value is set as " + retentionSize); + + if (localLogRetentionBytes > retentionSize) + throw new ConfigException(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localLogRetentionBytes, + "Value must not be more than property: " + TopicConfig.RETENTION_BYTES_CONFIG + " value."); + + this.localRetentionBytes = localLogRetentionBytes; + } + } + } + + // Visible for testing + public static class LogConfigDef extends ConfigDef { + public LogConfigDef() { + this(new ConfigDef()); + } + + public LogConfigDef(ConfigDef base) { + super(base); + } + + @Override + public List headers() { + return asList("Name", "Description", "Type", "Default", "Valid Values", SERVER_DEFAULT_HEADER_NAME, "Importance"); + } + + // Visible for testing + @Override + public String getConfigValue(ConfigKey key, String headerName) { + if (headerName.equals(SERVER_DEFAULT_HEADER_NAME)) + return ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.get(key.name); + else + return super.getConfigValue(key, headerName); + } + + public Optional serverConfigName(String configName) { + return Optional.ofNullable(ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.get(configName)); + } + } + + // Visible for testing + public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property"; + + public static final int DEFAULT_MAX_MESSAGE_BYTES = 1024 * 1024 + Records.LOG_OVERHEAD; + public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024; + 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"; + public static final long DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS = Long.MAX_VALUE; + public static final boolean DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE = true; + + public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false; + public static final int DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes + public static final int DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs + 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 */ + @Deprecated + public static final String DEFAULT_MESSAGE_FORMAT_VERSION = IBP_3_0_IV1.version(); + + // 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"; + public static final String FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG = "follower.replication.throttled.replicas"; + + @SuppressWarnings("deprecation") + private static final String MESSAGE_FORMAT_VERSION_CONFIG = TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG; + + // Visible for testing + public static final Set CONFIGS_WITH_NO_SERVER_DEFAULTS = Collections.unmodifiableSet(Utils.mkSet( + TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, + TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, + TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, + LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, + FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG + )); + + public static final String LEADER_REPLICATION_THROTTLED_REPLICAS_DOC = "A list of replicas for which log replication should be throttled on " + + "the leader side. The list should describe a set of replicas in the form " + + "[PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively the wildcard '*' can be used to throttle " + + "all replicas for this topic."; + public static final String FOLLOWER_REPLICATION_THROTTLED_REPLICAS_DOC = "A list of replicas for which log replication should be throttled on " + + "the follower side. The list should describe a set of " + "replicas in the form " + + "[PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively the wildcard '*' can be used to throttle " + + "all replicas for this topic."; + + @SuppressWarnings("deprecation") + private static final String MESSAGE_FORMAT_VERSION_DOC = TopicConfig.MESSAGE_FORMAT_VERSION_DOC; + + @SuppressWarnings("deprecation") + private static final LogConfigDef CONFIG = new LogConfigDef(); + static { + CONFIG. + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, + TopicConfig.SEGMENT_BYTES_DOC) + .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, + TopicConfig.SEGMENT_INDEX_BYTES_DOC) + .define(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, LONG, DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), MEDIUM, + TopicConfig.FLUSH_MESSAGES_INTERVAL_DOC) + .define(TopicConfig.FLUSH_MS_CONFIG, LONG, DEFAULT_FLUSH_MS, 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) + // 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, + TopicConfig.INDEX_INTERVAL_BYTES_DOC) + .define(TopicConfig.DELETE_RETENTION_MS_CONFIG, LONG, DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, + TopicConfig.DELETE_RETENTION_MS_DOC) + .define(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, LONG, DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, + 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, + 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, + 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, + 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, + in("CreateTime", "LogAppendTime"), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC) + .define(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS, + atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_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, + TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC) + .defineInternal(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_STORAGE_ENABLE, null, + MEDIUM, TopicConfig.REMOTE_LOG_STORAGE_ENABLE_DOC) + .defineInternal(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_MS, atLeast(-2), MEDIUM, + TopicConfig.LOCAL_LOG_RETENTION_MS_DOC) + .defineInternal(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, + TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC); + } + + public final Set overriddenConfigs; + + /* + * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig + * should also be in `KafkaConfig#extractLogConfigMap`. + */ + public final int segmentSize; + public final long segmentMs; + public final long segmentJitterMs; + public final int maxIndexSize; + public final long flushInterval; + public final long flushMs; + public final long retentionSize; + public final long retentionMs; + public final int indexInterval; + public final long fileDeleteDelayMs; + public final long deleteRetentionMs; + public final long compactionLagMs; + public final long maxCompactionLagMs; + public final double minCleanableRatio; + public final boolean compact; + public final boolean delete; + public final boolean uncleanLeaderElectionEnable; + public final int minInSyncReplicas; + public final String compressionType; + public final boolean preallocate; + + /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details regarding the deprecation */ + @Deprecated + public final MetadataVersion messageFormatVersion; + + public final TimestampType messageTimestampType; + public final long messageTimestampDifferenceMaxMs; + public final List leaderReplicationThrottledReplicas; + public final List followerReplicationThrottledReplicas; + public final boolean messageDownConversionEnable; + public final RemoteLogConfig remoteLogConfig; + + private final int maxMessageSize; + private final Map props; + + public LogConfig(Map props) { + this(props, Collections.emptySet()); + } + + @SuppressWarnings("deprecation") + public LogConfig(Map props, Set overriddenConfigs) { + super(CONFIG, props, false); + this.props = Collections.unmodifiableMap(props); + this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); + + this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); + this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); + this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); + this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); + this.flushInterval = getLong(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG); + this.flushMs = getLong(TopicConfig.FLUSH_MS_CONFIG); + this.retentionSize = getLong(TopicConfig.RETENTION_BYTES_CONFIG); + this.retentionMs = getLong(TopicConfig.RETENTION_MS_CONFIG); + this.maxMessageSize = getInt(TopicConfig.MAX_MESSAGE_BYTES_CONFIG); + this.indexInterval = getInt(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG); + this.fileDeleteDelayMs = getLong(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG); + this.deleteRetentionMs = getLong(TopicConfig.DELETE_RETENTION_MS_CONFIG); + this.compactionLagMs = getLong(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG); + this.maxCompactionLagMs = getLong(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG); + this.minCleanableRatio = getDouble(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG); + this.compact = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream() + .map(c -> c.toLowerCase(Locale.ROOT)) + .collect(Collectors.toList()) + .contains(TopicConfig.CLEANUP_POLICY_COMPACT); + this.delete = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream() + .map(c -> c.toLowerCase(Locale.ROOT)) + .collect(Collectors.toList()) + .contains(TopicConfig.CLEANUP_POLICY_DELETE); + this.uncleanLeaderElectionEnable = getBoolean(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG); + this.minInSyncReplicas = getInt(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG); + this.compressionType = getString(TopicConfig.COMPRESSION_TYPE_CONFIG).toLowerCase(Locale.ROOT); + this.preallocate = getBoolean(TopicConfig.PREALLOCATE_CONFIG); + this.messageFormatVersion = MetadataVersion.fromVersionString(getString(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)); + this.messageTimestampType = TimestampType.forName(getString(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG)); + this.messageTimestampDifferenceMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG); + this.leaderReplicationThrottledReplicas = Collections.unmodifiableList(getList(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG)); + this.followerReplicationThrottledReplicas = Collections.unmodifiableList(getList(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG)); + this.messageDownConversionEnable = getBoolean(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG); + + remoteLogConfig = new RemoteLogConfig(this, retentionMs, retentionSize); + } + + @SuppressWarnings("deprecation") + public RecordVersion recordVersion() { + return messageFormatVersion.highestSupportedRecordVersion(); + } + + // Exposed as a method so it can be mocked + public int maxMessageSize() { + return maxMessageSize; + } + + public long randomSegmentJitter() { + if (segmentJitterMs == 0) + return 0; + else + return Utils.abs(ThreadLocalRandom.current().nextInt()) % Math.min(segmentJitterMs, segmentMs); + } + + public long maxSegmentMs() { + if (compact && maxCompactionLagMs > 0) + return Math.min(maxCompactionLagMs, segmentMs); + else + return segmentMs; + } + + public int initFileSize() { + if (preallocate) + return segmentSize; + else + return 0; + } + + public String overriddenConfigsAsLoggableString() { + Map overriddenTopicProps = new HashMap<>(); + props.forEach((k, v) -> { + if (overriddenConfigs.contains(k)) + overriddenTopicProps.put((String) k, v); + }); + return ConfigUtils.configMapToRedactedString(overriddenTopicProps, CONFIG); + } + + /** + * Create a log config instance using the given properties and defaults + */ + public static LogConfig fromProps(Map defaults, Properties overrides) { + Properties props = new Properties(); + defaults.forEach((k, v) -> props.put(k, v)); + props.putAll(overrides); + Set overriddenKeys = overrides.keySet().stream().map(k -> (String) k).collect(Collectors.toSet()); + return new LogConfig(props, overriddenKeys); + } + + // Visible for testing, return a copy since it's a mutable global variable + public static LogConfigDef configDefCopy() { + return new LogConfigDef(CONFIG); + } + + public static boolean shouldIgnoreMessageFormatVersion(MetadataVersion interBrokerProtocolVersion) { + return interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1); + } + + public static Optional configType(String configName) { + return Optional.ofNullable(CONFIG.configKeys().get(configName)).map(c -> c.type); + } + + public static List configNames() { + return CONFIG.names().stream().sorted().collect(Collectors.toList()); + } + + public static Optional serverConfigName(String configName) { + return CONFIG.serverConfigName(configName); + } + + public static Map configKeys() { + return Collections.unmodifiableMap(CONFIG.configKeys()); + } + + /** + * Check that property names are valid + */ + public static void validateNames(Properties props) { + List names = configNames(); + for (Object name : props.keySet()) + if (!names.contains(name)) + throw new InvalidConfigurationException("Unknown topic config name: " + name); + } + + public static void validateValues(Map props) { + long minCompactionLag = (Long) props.get(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG); + long maxCompactionLag = (Long) props.get(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG); + if (minCompactionLag > maxCompactionLag) { + throw new InvalidConfigurationException("conflict topic config setting " + + TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG + " (" + minCompactionLag + ") > " + + TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG + " (" + maxCompactionLag + ")"); + } + } + + /** + * Check that the given properties contain only valid log config names and that all values can be parsed and are valid + */ + public static void validate(Properties props) { + validateNames(props); + Map valueMaps = CONFIG.parse(props); + validateValues(valueMaps); + } + + public static void main(String[] args) { + System.out.println(CONFIG.toHtml(4, config -> "topicconfigs_" + config)); + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/ThrottledReplicaListValidator.java b/storage/src/main/java/org/apache/kafka/server/log/internals/ThrottledReplicaListValidator.java new file mode 100644 index 0000000000000..a98e7f747c1f3 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/ThrottledReplicaListValidator.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.internals; + +import static java.util.Arrays.asList; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.kafka.common.config.ConfigDef.Validator; +import org.apache.kafka.common.config.ConfigException; + +public class ThrottledReplicaListValidator implements Validator { + public static final Validator INSTANCE = new ThrottledReplicaListValidator(); + + private ThrottledReplicaListValidator() { } + + public static void ensureValidString(String name, String value) { + INSTANCE.ensureValid(name, asList(value.split(","))); + } + + @Override + public void ensureValid(String name, Object value) { + if (value instanceof java.util.List) { + List proposed = ((List) value).stream().map(element -> element.toString().trim()).collect(Collectors.toList()); + if (!(proposed.stream().allMatch(s -> s.matches("([0-9]+:[0-9]+)?")) + || String.join("", proposed).equals("*"))) + throw new ConfigException(name, value, name + + " must be the literal '*' or a list of replicas in the following format: [partitionId]:[brokerId],[partitionId]:[brokerId],..."); + } else + throw new ConfigException(name, value, name + " must be a List but was " + value.getClass().getName()); + } + + @Override + public String toString() { + return "[partitionId]:[brokerId],[partitionId]:[brokerId],..."; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 37c26a6a4b97e..0211172864545 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.integration; import java.time.Duration; -import kafka.log.LogConfig; import kafka.utils.MockTime; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -26,6 +25,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; @@ -210,10 +210,10 @@ public void shouldCompactTopicsForKeyValueStoreChangelogs() { streams.close(); final Properties changelogProps = getTopicProperties(ProcessorStateManager.storeChangelogTopic(appID, "Counts", null)); - assertEquals(LogConfig.Compact(), changelogProps.getProperty(LogConfig.CleanupPolicyProp())); + assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, changelogProps.getProperty(TopicConfig.CLEANUP_POLICY_CONFIG)); final Properties repartitionProps = getTopicProperties(appID + "-Counts-repartition"); - assertEquals(LogConfig.Delete(), repartitionProps.getProperty(LogConfig.CleanupPolicyProp())); + assertEquals(TopicConfig.CLEANUP_POLICY_DELETE, repartitionProps.getProperty(TopicConfig.CLEANUP_POLICY_CONFIG)); assertEquals(4, repartitionProps.size()); } @@ -249,16 +249,16 @@ public void shouldCompactAndDeleteTopicsForWindowStoreChangelogs() { waitForCompletion(streams, 2, 30000L); streams.close(); final Properties properties = getTopicProperties(ProcessorStateManager.storeChangelogTopic(appID, "CountWindows", null)); - final List policies = Arrays.asList(properties.getProperty(LogConfig.CleanupPolicyProp()).split(",")); + final List policies = Arrays.asList(properties.getProperty(TopicConfig.CLEANUP_POLICY_CONFIG).split(",")); assertEquals(2, policies.size()); - assertTrue(policies.contains(LogConfig.Compact())); - assertTrue(policies.contains(LogConfig.Delete())); + assertTrue(policies.contains(TopicConfig.CLEANUP_POLICY_COMPACT)); + assertTrue(policies.contains(TopicConfig.CLEANUP_POLICY_DELETE)); // retention should be 1 day + the window duration final long retention = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS) + durationMs; - assertEquals(retention, Long.parseLong(properties.getProperty(LogConfig.RetentionMsProp()))); + assertEquals(retention, Long.parseLong(properties.getProperty(TopicConfig.RETENTION_MS_CONFIG))); final Properties repartitionProps = getTopicProperties(appID + "-CountWindows-repartition"); - assertEquals(LogConfig.Delete(), repartitionProps.getProperty(LogConfig.CleanupPolicyProp())); + assertEquals(TopicConfig.CLEANUP_POLICY_DELETE, repartitionProps.getProperty(TopicConfig.CLEANUP_POLICY_CONFIG)); assertEquals(4, repartitionProps.size()); } }