KAFKA-15853 Move KafkaConfig log properties and docs out of core (#15569)

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Nikolay <nizhikov@apache.org>, Federico Valeri <fvaleri@redhat.com>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Omnia Ibrahim 2024-04-19 21:14:23 +01:00 committed by GitHub
parent 76e089142b
commit ecb2dd4cdc
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
67 changed files with 602 additions and 518 deletions

View File

@ -111,9 +111,13 @@
<allow class="org.apache.kafka.server.util.ShutdownableThread" /> <allow class="org.apache.kafka.server.util.ShutdownableThread" />
</subpackage> </subpackage>
</subpackage> </subpackage>
<subpackage name="config">
<allow pkg="org.apache.kafka.server"/>
</subpackage>
</subpackage> </subpackage>
<subpackage name="admin"> <subpackage name="admin">
<allow pkg="org.apache.kafka.server.common" /> <allow pkg="org.apache.kafka.server.common" />
</subpackage> </subpackage>
</import-control> </import-control>

View File

@ -94,6 +94,8 @@ import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CL
import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG;
import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG; import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG;
import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG; import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG;
import static org.apache.kafka.server.config.ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG;
import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
/** /**
* Setup an embedded Kafka cluster with specified number of brokers and specified broker properties. To be used for * Setup an embedded Kafka cluster with specified number of brokers and specified broker properties. To be used for
@ -163,7 +165,7 @@ public class EmbeddedKafkaCluster {
putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true); putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true);
putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0);
putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) brokers.length); putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) brokers.length);
putIfAbsent(brokerConfig, KafkaConfig.AutoCreateTopicsEnableProp(), false); putIfAbsent(brokerConfig, AUTO_CREATE_TOPICS_ENABLE_CONFIG, false);
// reduce the size of the log cleaner map to reduce test memory usage // reduce the size of the log cleaner map to reduce test memory usage
putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L); putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L);
@ -177,7 +179,7 @@ public class EmbeddedKafkaCluster {
for (int i = 0; i < brokers.length; i++) { for (int i = 0; i < brokers.length; i++) {
brokerConfig.put(KafkaConfig.BrokerIdProp(), i); brokerConfig.put(KafkaConfig.BrokerIdProp(), i);
currentBrokerLogDirs[i] = currentBrokerLogDirs[i] == null ? createLogDir() : currentBrokerLogDirs[i]; currentBrokerLogDirs[i] = currentBrokerLogDirs[i] == null ? createLogDir() : currentBrokerLogDirs[i];
brokerConfig.put(KafkaConfig.LogDirProp(), currentBrokerLogDirs[i]); brokerConfig.put(LOG_DIR_CONFIG, currentBrokerLogDirs[i]);
if (!hasListenerConfig) if (!hasListenerConfig)
brokerConfig.put(KafkaConfig.ListenersProp(), listenerName.value() + "://localhost:" + currentBrokerPorts[i]); brokerConfig.put(KafkaConfig.ListenersProp(), listenerName.value() + "://localhost:" + currentBrokerPorts[i]);
brokers[i] = TestUtils.createServer(new KafkaConfig(brokerConfig, true), time); brokers[i] = TestUtils.createServer(new KafkaConfig(brokerConfig, true), time);

View File

@ -120,6 +120,7 @@ import java.util.stream.Stream;
import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX; import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX;
import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC; import static org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics.REMOTE_LOG_MANAGER_TASKS_AVG_IDLE_PERCENT_METRIC;
import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
/** /**
* This class is responsible for * This class is responsible for
@ -282,7 +283,7 @@ public class RemoteLogManager implements Closeable {
rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps()); rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps());
rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId); rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId);
rlmmProps.put(KafkaConfig.LogDirProp(), logDir); rlmmProps.put(LOG_DIR_CONFIG, logDir);
rlmmProps.put("cluster.id", clusterId); rlmmProps.put("cluster.id", clusterId);
remoteLogMetadataManager.configure(rlmmProps); remoteLogMetadataManager.configure(rlmmProps);

View File

@ -18,7 +18,7 @@ package kafka.raft
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import org.apache.kafka.common.config.AbstractConfig import org.apache.kafka.common.config.AbstractConfig
import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.server.config.ServerLogConfigs
final case class MetadataLogConfig( final case class MetadataLogConfig(
logSegmentBytes: Int, logSegmentBytes: Int,
@ -42,7 +42,7 @@ object MetadataLogConfig {
config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp), config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp),
maxBatchSizeInBytes, maxBatchSizeInBytes,
maxFetchSizeInBytes, maxFetchSizeInBytes,
LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
config.getInt(KafkaConfig.NodeIdProp) config.getInt(KafkaConfig.NodeIdProp)
) )
} }

View File

@ -31,6 +31,7 @@ import org.apache.kafka.common.record.{MemoryRecords, Records}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} 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.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch}
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.Scheduler import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals import org.apache.kafka.storage.internals
@ -553,7 +554,7 @@ object KafkaMetadataLog extends Logging {
props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString)
props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString)
props.setProperty(TopicConfig.SEGMENT_MS_CONFIG, config.logSegmentMillis.toString) props.setProperty(TopicConfig.SEGMENT_MS_CONFIG, config.logSegmentMillis.toString)
props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS.toString) props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString)
// Disable time and byte retention when deleting segments // Disable time and byte retention when deleting segments
props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1") props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1")

View File

@ -21,7 +21,6 @@ import kafka.metrics.LinuxIoMetricsCollector
import kafka.migration.MigrationPropagator import kafka.migration.MigrationPropagator
import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.raft.KafkaRaftManager import kafka.raft.KafkaRaftManager
import kafka.server.KafkaConfig.{AlterConfigPolicyClassNameProp, CreateTopicPolicyClassNameProp}
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
import scala.collection.immutable import scala.collection.immutable
@ -46,6 +45,7 @@ import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.security.{CredentialProvider, PasswordEncoder} import org.apache.kafka.security.{CredentialProvider, PasswordEncoder}
import org.apache.kafka.server.NodeToControllerChannelManager import org.apache.kafka.server.NodeToControllerChannelManager
import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.config.ServerLogConfigs.{ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG}
import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.server.config.ConfigType import org.apache.kafka.server.config.ConfigType
import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
@ -207,9 +207,9 @@ class ControllerServer(
sharedServer.startForController() sharedServer.startForController()
createTopicPolicy = Option(config. createTopicPolicy = Option(config.
getConfiguredInstance(CreateTopicPolicyClassNameProp, classOf[CreateTopicPolicy])) getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy]))
alterConfigPolicy = Option(config. alterConfigPolicy = Option(config.
getConfiguredInstance(AlterConfigPolicyClassNameProp, classOf[AlterConfigPolicy])) getConfiguredInstance(ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[AlterConfigPolicy]))
val voterConnections = FutureUtils.waitWithLogging(logger.underlying, logIdent, val voterConnections = FutureUtils.waitWithLogging(logger.underlying, logIdent,
"controller quorum voters future", "controller quorum voters future",

View File

@ -39,8 +39,8 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals}
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.ClientMetricsReceiverPlugin import org.apache.kafka.server.metrics.ClientMetricsReceiverPlugin
import org.apache.kafka.server.telemetry.ClientTelemetry import org.apache.kafka.server.telemetry.ClientTelemetry
import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig}
@ -120,14 +120,14 @@ object DynamicBrokerConfig {
def brokerConfigSynonyms(name: String, matchListenerOverride: Boolean): List[String] = { def brokerConfigSynonyms(name: String, matchListenerOverride: Boolean): List[String] = {
name match { name match {
case KafkaConfig.LogRollTimeMillisProp | KafkaConfig.LogRollTimeHoursProp => case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG | ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG =>
List(KafkaConfig.LogRollTimeMillisProp, KafkaConfig.LogRollTimeHoursProp) List(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG)
case KafkaConfig.LogRollTimeJitterMillisProp | KafkaConfig.LogRollTimeJitterHoursProp => case ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG | ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG =>
List(KafkaConfig.LogRollTimeJitterMillisProp, KafkaConfig.LogRollTimeJitterHoursProp) List(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG)
case KafkaConfig.LogFlushIntervalMsProp => // LogFlushSchedulerIntervalMsProp is used as default case ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG => // KafkaLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG is used as default
List(KafkaConfig.LogFlushIntervalMsProp, KafkaConfig.LogFlushSchedulerIntervalMsProp) List(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG)
case KafkaConfig.LogRetentionTimeMillisProp | KafkaConfig.LogRetentionTimeMinutesProp | KafkaConfig.LogRetentionTimeHoursProp => case ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG | ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG | ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG =>
List(KafkaConfig.LogRetentionTimeMillisProp, KafkaConfig.LogRetentionTimeMinutesProp, KafkaConfig.LogRetentionTimeHoursProp) List(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG)
case ListenerConfigRegex(baseName) if matchListenerOverride => case ListenerConfigRegex(baseName) if matchListenerOverride =>
// `ListenerMechanismConfigs` are specified as listenerPrefix.mechanism.<configName> // `ListenerMechanismConfigs` are specified as listenerPrefix.mechanism.<configName>
// and other listener configs are specified as listenerPrefix.<configName> // and other listener configs are specified as listenerPrefix.<configName>
@ -674,7 +674,7 @@ object DynamicLogConfig {
// Exclude message.format.version for now since we need to check that the version // Exclude message.format.version for now since we need to check that the version
// is supported on all brokers in the cluster. // is supported on all brokers in the cluster.
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
val ExcludedConfigs = Set(KafkaConfig.LogMessageFormatVersionProp) val ExcludedConfigs = Set(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG)
val ReconfigurableConfigs = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.values.asScala.toSet -- ExcludedConfigs val ReconfigurableConfigs = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.values.asScala.toSet -- ExcludedConfigs
val KafkaConfigToLogConfigName = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) } val KafkaConfigToLogConfigName = ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) }
@ -697,11 +697,11 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok
if (logRetentionMs != -1L && logLocalRetentionMs != -2L) { if (logRetentionMs != -1L && logLocalRetentionMs != -2L) {
if (logLocalRetentionMs == -1L) { if (logLocalRetentionMs == -1L) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs, throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs,
s"Value must not be -1 as ${KafkaConfig.LogRetentionTimeMillisProp} value is set as $logRetentionMs.") s"Value must not be -1 as ${ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG} value is set as $logRetentionMs.")
} }
if (logLocalRetentionMs > logRetentionMs) { if (logLocalRetentionMs > logRetentionMs) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs, throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, logLocalRetentionMs,
s"Value must not be more than ${KafkaConfig.LogRetentionTimeMillisProp} property value: $logRetentionMs") s"Value must not be more than ${ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG} property value: $logRetentionMs")
} }
} }
} }
@ -712,11 +712,11 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok
if (logRetentionBytes > -1 && logLocalRetentionBytes != -2) { if (logRetentionBytes > -1 && logLocalRetentionBytes != -2) {
if (logLocalRetentionBytes == -1) { if (logLocalRetentionBytes == -1) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes, throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes,
s"Value must not be -1 as ${KafkaConfig.LogRetentionBytesProp} value is set as $logRetentionBytes.") s"Value must not be -1 as ${ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG} value is set as $logRetentionBytes.")
} }
if (logLocalRetentionBytes > logRetentionBytes) { if (logLocalRetentionBytes > logRetentionBytes) {
throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes, throw new ConfigException(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, logLocalRetentionBytes,
s"Value must not be more than ${KafkaConfig.LogRetentionBytesProp} property value: $logRetentionBytes") s"Value must not be more than ${ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG} property value: $logRetentionBytes")
} }
} }
} }
@ -771,7 +771,7 @@ object DynamicThreadPool {
val ReconfigurableConfigs = Set( val ReconfigurableConfigs = Set(
KafkaConfig.NumIoThreadsProp, KafkaConfig.NumIoThreadsProp,
ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG,
KafkaConfig.NumRecoveryThreadsPerDataDirProp, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG,
KafkaConfig.BackgroundThreadsProp) KafkaConfig.BackgroundThreadsProp)
def validateReconfiguration(currentConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { def validateReconfiguration(currentConfig: KafkaConfig, newConfig: KafkaConfig): Unit = {
@ -796,7 +796,7 @@ object DynamicThreadPool {
name match { name match {
case KafkaConfig.NumIoThreadsProp => config.numIoThreads case KafkaConfig.NumIoThreadsProp => config.numIoThreads
case ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG => config.numReplicaFetchers case ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG => config.numReplicaFetchers
case KafkaConfig.NumRecoveryThreadsPerDataDirProp => config.numRecoveryThreadsPerDataDir case ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG => config.numRecoveryThreadsPerDataDir
case KafkaConfig.BackgroundThreadsProp => config.backgroundThreads case KafkaConfig.BackgroundThreadsProp => config.backgroundThreads
case n => throw new IllegalStateException(s"Unexpected config $n") case n => throw new IllegalStateException(s"Unexpected config $n")
} }

View File

@ -47,8 +47,7 @@ import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator} import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator}
import org.apache.kafka.server.common.MetadataVersion._ import org.apache.kafka.server.common.MetadataVersion._
import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.record.BrokerCompressionType
import org.apache.kafka.server.util.Csv import org.apache.kafka.server.util.Csv
@ -63,8 +62,6 @@ import scala.collection.{Map, Seq}
object KafkaConfig { object KafkaConfig {
private val LogConfigPrefix = "log."
def main(args: Array[String]): Unit = { def main(args: Array[String]): Unit = {
System.out.println(configDef.toHtml(4, (config: String) => "brokerconfigs_" + config, System.out.println(configDef.toHtml(4, (config: String) => "brokerconfigs_" + config,
DynamicBrokerConfig.dynamicConfigUpdateModes)) DynamicBrokerConfig.dynamicConfigUpdateModes))
@ -157,54 +154,7 @@ object KafkaConfig {
val FailedAuthenticationDelayMsProp = "connection.failed.authentication.delay.ms" val FailedAuthenticationDelayMsProp = "connection.failed.authentication.delay.ms"
/***************** rack configuration *************/ /***************** rack configuration *************/
val RackProp = "broker.rack" val RackProp = "broker.rack"
/** ********* Log Configuration ***********/
val NumPartitionsProp = "num.partitions"
val LogDirsProp = LogConfigPrefix + "dirs"
val LogDirProp = LogConfigPrefix + "dir"
val LogSegmentBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG)
val LogRollTimeMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG)
val LogRollTimeHoursProp = LogConfigPrefix + "roll.hours"
val LogRollTimeJitterMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_JITTER_MS_CONFIG)
val LogRollTimeJitterHoursProp = LogConfigPrefix + "roll.jitter.hours"
val LogRetentionTimeMillisProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_MS_CONFIG)
val LogRetentionTimeMinutesProp = LogConfigPrefix + "retention.minutes"
val LogRetentionTimeHoursProp = LogConfigPrefix + "retention.hours"
val LogRetentionBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_BYTES_CONFIG)
val LogCleanupIntervalMsProp = LogConfigPrefix + "retention.check.interval.ms"
val LogCleanupPolicyProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG)
val LogIndexSizeMaxBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG)
val LogIndexIntervalBytesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG)
val LogFlushIntervalMessagesProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG)
val LogDeleteDelayMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG)
val LogFlushSchedulerIntervalMsProp = LogConfigPrefix + "flush.scheduler.interval.ms"
val LogFlushIntervalMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MS_CONFIG)
val LogFlushOffsetCheckpointIntervalMsProp = LogConfigPrefix + "flush.offset.checkpoint.interval.ms"
val LogFlushStartOffsetCheckpointIntervalMsProp = LogConfigPrefix + "flush.start.offset.checkpoint.interval.ms"
val LogPreAllocateProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.PREALLOCATE_CONFIG)
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
@deprecated("3.0")
val LogMessageFormatVersionProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)
val LogMessageTimestampTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG)
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
@deprecated("3.6")
val LogMessageTimestampDifferenceMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG)
val LogMessageTimestampBeforeMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)
val LogMessageTimestampAfterMaxMsProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)
val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir"
val AutoCreateTopicsEnableProp = "auto.create.topics.enable"
val MinInSyncReplicasProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG)
val CreateTopicPolicyClassNameProp = "create.topic.policy.class.name"
val AlterConfigPolicyClassNameProp = "alter.config.policy.class.name"
val LogMessageDownConversionEnableProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
/** ********* Controlled shutdown configuration ***********/ /** ********* Controlled shutdown configuration ***********/
val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
@ -405,75 +355,6 @@ object KafkaConfig {
s"This must be configured to be less than $ConnectionsMaxIdleMsProp to prevent connection timeout." s"This must be configured to be less than $ConnectionsMaxIdleMsProp to prevent connection timeout."
/************* Rack Configuration **************/ /************* Rack Configuration **************/
val RackDoc = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: <code>RACK1</code>, <code>us-east-1d</code>" val RackDoc = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: <code>RACK1</code>, <code>us-east-1d</code>"
/** ********* Log Configuration ***********/
val NumPartitionsDoc = "The default number of log partitions per topic"
val LogDirDoc = "The directory in which the log data is kept (supplemental for " + LogDirsProp + " property)"
val LogDirsDoc = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LogDirProp + " is used."
val LogSegmentBytesDoc = "The maximum size of a single log file"
val LogRollTimeMillisDoc = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LogRollTimeHoursProp + " is used"
val LogRollTimeHoursDoc = "The maximum time before a new log segment is rolled out (in hours), secondary to " + LogRollTimeMillisProp + " property"
val LogRollTimeJitterMillisDoc = "The maximum jitter to subtract from logRollTimeMillis (in milliseconds). If not set, the value in " + LogRollTimeJitterHoursProp + " is used"
val LogRollTimeJitterHoursDoc = "The maximum jitter to subtract from logRollTimeMillis (in hours), secondary to " + LogRollTimeJitterMillisProp + " property"
val LogRetentionTimeMillisDoc = "The number of milliseconds to keep a log file before deleting it (in milliseconds), If not set, the value in " + LogRetentionTimeMinutesProp + " is used. If set to -1, no time limit is applied."
val LogRetentionTimeMinsDoc = "The number of minutes to keep a log file before deleting it (in minutes), secondary to " + LogRetentionTimeMillisProp + " property. If not set, the value in " + LogRetentionTimeHoursProp + " is used"
val LogRetentionTimeHoursDoc = "The number of hours to keep a log file before deleting it (in hours), tertiary to " + LogRetentionTimeMillisProp + " property"
val LogRetentionBytesDoc = "The maximum size of the log before deleting it"
val LogCleanupIntervalMsDoc = "The frequency in milliseconds that the log cleaner checks whether any log is eligible for deletion"
val LogCleanupPolicyDoc = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies. Valid policies are: \"delete\" and \"compact\""
val LogIndexSizeMaxBytesDoc = "The maximum size in bytes of the offset index"
val LogIndexIntervalBytesDoc = "The interval with which we add an entry to the offset index."
val LogFlushIntervalMessagesDoc = "The number of messages accumulated on a log partition before messages are flushed to disk."
val LogDeleteDelayMsDoc = "The amount of time to wait before deleting a file from the filesystem"
val LogFlushSchedulerIntervalMsDoc = "The frequency in ms that the log flusher checks whether any log needs to be flushed to disk"
val LogFlushIntervalMsDoc = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk. If not set, the value in " + LogFlushSchedulerIntervalMsProp + " is used"
val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point."
val LogFlushStartOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of log start offset"
val LogPreAllocateEnableDoc = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true."
val LogMessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid MetadataVersion. " +
"Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check MetadataVersion for more details. By setting a particular message format version, the " +
"user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " +
"will cause consumers with older versions to break as they will receive messages with a format that they don't understand."
val LogMessageTimestampTypeDoc = "Define whether the timestamp in the message is message create time or log append time. The value should be either " +
"<code>CreateTime</code> or <code>LogAppendTime</code>."
val LogMessageTimestampDifferenceMaxMsDoc = "[DEPRECATED] The maximum difference allowed between the timestamp when a broker receives " +
"a message and the timestamp specified in the message. If log.message.timestamp.type=CreateTime, a message will be rejected " +
"if the difference in timestamp exceeds this threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." +
"The maximum timestamp difference allowed should be no greater than log.retention.ms to avoid unnecessarily frequent log rolling."
val LogMessageTimestampBeforeMaxMsDoc = "This configuration sets the allowable timestamp difference between the " +
"broker's timestamp and the message timestamp. The message timestamp can be earlier than or equal to the broker's " +
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime."
val LogMessageTimestampAfterMaxMsDoc = "This configuration sets the allowable timestamp difference between the " +
"message timestamp and the broker's timestamp. The message timestamp can be later than or equal to the broker's " +
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime."
val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown"
val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server."
val MinInSyncReplicasDoc = "When a producer sets acks to \"all\" (or \"-1\"), " +
"<code>min.insync.replicas</code> specifies the minimum number of replicas that must acknowledge " +
"a write for the write to be considered successful. If this minimum cannot be met, " +
"then the producer will raise an exception (either <code>NotEnoughReplicas</code> or " +
"<code>NotEnoughReplicasAfterAppend</code>).<br>When used together, <code>min.insync.replicas</code> and acks " +
"allow you to enforce greater durability guarantees. A typical scenario would be to " +
"create a topic with a replication factor of 3, set <code>min.insync.replicas</code> to 2, and " +
"produce with acks of \"all\". This will ensure that the producer raises an exception " +
"if a majority of replicas do not receive a write."
val CreateTopicPolicyClassNameDoc = "The create topic policy class that should be used for validation. The class should " +
"implement the <code>org.apache.kafka.server.policy.CreateTopicPolicy</code> interface."
val AlterConfigPolicyClassNameDoc = "The alter configs policy class that should be used for validation. The class should " +
"implement the <code>org.apache.kafka.server.policy.AlterConfigPolicy</code> interface."
val LogMessageDownConversionEnableDoc = TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC
/** ********* Controlled shutdown configuration ***********/ /** ********* Controlled shutdown configuration ***********/
val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens"
@ -643,24 +524,24 @@ object KafkaConfig {
.define(RackProp, STRING, null, MEDIUM, RackDoc) .define(RackProp, STRING, null, MEDIUM, RackDoc)
/** ********* Log Configuration ***********/ /** ********* Log Configuration ***********/
.define(NumPartitionsProp, INT, Defaults.NUM_PARTITIONS, atLeast(1), MEDIUM, NumPartitionsDoc) .define(ServerLogConfigs.NUM_PARTITIONS_CONFIG, INT, ServerLogConfigs.NUM_PARTITIONS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.NUM_PARTITIONS_DOC)
.define(LogDirProp, STRING, Defaults.LOG_DIR, HIGH, LogDirDoc) .define(ServerLogConfigs.LOG_DIR_CONFIG, STRING, ServerLogConfigs.LOG_DIR_DEFAULT, HIGH, ServerLogConfigs.LOG_DIR_DOC)
.define(LogDirsProp, STRING, null, HIGH, LogDirsDoc) .define(ServerLogConfigs.LOG_DIRS_CONFIG, STRING, null, HIGH, ServerLogConfigs.LOG_DIRS_DOC)
.define(LogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, LogSegmentBytesDoc) .define(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC)
.define(LogRollTimeMillisProp, LONG, null, HIGH, LogRollTimeMillisDoc) .define(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_ROLL_TIME_MILLIS_DOC)
.define(LogRollTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_MS).toInt, atLeast(1), HIGH, LogRollTimeHoursDoc) .define(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_MS).toInt, atLeast(1), HIGH, ServerLogConfigs.LOG_ROLL_TIME_HOURS_DOC)
.define(LogRollTimeJitterMillisProp, LONG, null, HIGH, LogRollTimeJitterMillisDoc) .define(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_DOC)
.define(LogRollTimeJitterHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_JITTER_MS).toInt, atLeast(0), HIGH, LogRollTimeJitterHoursDoc) .define(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_SEGMENT_JITTER_MS).toInt, atLeast(0), HIGH, ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_DOC)
.define(LogRetentionTimeMillisProp, LONG, null, HIGH, LogRetentionTimeMillisDoc) .define(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_DOC)
.define(LogRetentionTimeMinutesProp, INT, null, HIGH, LogRetentionTimeMinsDoc) .define(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, INT, null, HIGH, ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_DOC)
.define(LogRetentionTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_RETENTION_MS).toInt, HIGH, LogRetentionTimeHoursDoc) .define(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_RETENTION_MS).toInt, HIGH, ServerLogConfigs.LOG_RETENTION_TIME_HOURS_DOC)
.define(LogRetentionBytesProp, LONG, LogConfig.DEFAULT_RETENTION_BYTES, HIGH, LogRetentionBytesDoc) .define(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, LONG, ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, HIGH, ServerLogConfigs.LOG_RETENTION_BYTES_DOC)
.define(LogCleanupIntervalMsProp, LONG, Defaults.LOG_CLEANUP_INTERVAL_MS, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc) .define(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG, LONG, ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_DOC)
.define(LogCleanupPolicyProp, LIST, LogConfig.DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, LogCleanupPolicyDoc) .define(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG, LIST, ServerLogConfigs.LOG_CLEANUP_POLICY_DEFAULT, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, ServerLogConfigs.LOG_CLEANUP_POLICY_DOC)
.define(CleanerConfig.LOG_CLEANER_THREADS_PROP, INT, CleanerConfig.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_THREADS_DOC) .define(CleanerConfig.LOG_CLEANER_THREADS_PROP, INT, CleanerConfig.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_THREADS_DOC)
.define(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, DOUBLE, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_DOC) .define(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, DOUBLE, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_DOC)
.define(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, LONG, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_DOC) .define(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, LONG, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_DOC)
@ -672,26 +553,26 @@ object KafkaConfig {
.define(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_DOC) .define(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_DOC)
.define(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_DOC) .define(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_DOC)
.define(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_DOC) .define(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_DOC)
.define(LogIndexSizeMaxBytesProp, INT, LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc) .define(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT, atLeast(4), MEDIUM, ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DOC)
.define(LogIndexIntervalBytesProp, INT, LogConfig.DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc) .define(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DOC)
.define(LogFlushIntervalMessagesProp, LONG, LogConfig.DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), HIGH, LogFlushIntervalMessagesDoc) .define(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_DOC)
.define(LogDeleteDelayMsProp, LONG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), HIGH, LogDeleteDelayMsDoc) .define(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, atLeast(0), HIGH, ServerLogConfigs.LOG_DELETE_DELAY_MS_DOC)
.define(LogFlushSchedulerIntervalMsProp, LONG, LogConfig.DEFAULT_FLUSH_MS, HIGH, LogFlushSchedulerIntervalMsDoc) .define(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_DEFAULT, HIGH, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_DOC)
.define(LogFlushIntervalMsProp, LONG, null, HIGH, LogFlushIntervalMsDoc) .define(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_DOC)
.define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) .define(ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG, INT, ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DOC)
.define(LogFlushStartOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushStartOffsetCheckpointIntervalMsDoc) .define(ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG, INT, ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DOC)
.define(LogPreAllocateProp, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, LogPreAllocateEnableDoc) .define(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, ServerLogConfigs.LOG_PRE_ALLOCATE_ENABLE_DOC)
.define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NUM_RECOVERY_THREADS_PER_DATA_DIR, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) .define(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, INT, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_DOC)
.define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AUTO_CREATE_TOPICS_ENABLE, HIGH, AutoCreateTopicsEnableDoc) .define(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DEFAULT, HIGH, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_DOC)
.define(MinInSyncReplicasProp, INT, LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS, atLeast(1), HIGH, MinInSyncReplicasDoc) .define(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, INT, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT, atLeast(1), HIGH, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DOC)
.define(LogMessageFormatVersionProp, STRING, LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM, LogMessageFormatVersionDoc) .define(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT, new MetadataVersionValidator(), MEDIUM, ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DOC)
.define(LogMessageTimestampTypeProp, STRING, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_TYPE, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT, ConfigDef.ValidString.in("CreateTime", "LogAppendTime"), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DOC)
.define(LogMessageTimestampDifferenceMaxMsProp, LONG, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
.define(LogMessageTimestampBeforeMaxMsProp, LONG, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_BEFORE_MAX_MS, atLeast(0), MEDIUM, LogMessageTimestampBeforeMaxMsDoc) .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC)
.define(LogMessageTimestampAfterMaxMsProp, LONG, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_AFTER_MAX_MS, atLeast(0), MEDIUM, LogMessageTimestampAfterMaxMsDoc) .define(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT, atLeast(0), MEDIUM, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC)
.define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc) .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC)
.define(AlterConfigPolicyClassNameProp, CLASS, null, LOW, AlterConfigPolicyClassNameDoc) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC)
.define(LogMessageDownConversionEnableProp, BOOLEAN, LogConfig.DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, LogMessageDownConversionEnableDoc) .define(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT, LOW, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DOC)
/** ********* Replication configuration ***********/ /** ********* Replication configuration ***********/
.define(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, INT, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DEFAULT, MEDIUM, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DOC) .define(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, INT, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DEFAULT, MEDIUM, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DOC)
@ -1235,21 +1116,22 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG)) val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG))
/** ********* Log Configuration ***********/ /** ********* Log Configuration ***********/
val autoCreateTopicsEnable = getBoolean(KafkaConfig.AutoCreateTopicsEnableProp) val autoCreateTopicsEnable = getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG)
val numPartitions = getInt(KafkaConfig.NumPartitionsProp) val numPartitions = getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG)
val logDirs = CoreUtils.parseCsvList(Option(getString(KafkaConfig.LogDirsProp)).getOrElse(getString(KafkaConfig.LogDirProp))) val logDirs = CoreUtils.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG)))
def logSegmentBytes = getInt(KafkaConfig.LogSegmentBytesProp) def logSegmentBytes = getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG)
def logFlushIntervalMessages = getLong(KafkaConfig.LogFlushIntervalMessagesProp) def logFlushIntervalMessages = getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG)
val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP) val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP)
def numRecoveryThreadsPerDataDir = getInt(KafkaConfig.NumRecoveryThreadsPerDataDirProp) def numRecoveryThreadsPerDataDir = getInt(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG)
val logFlushSchedulerIntervalMs = getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp) val logFlushSchedulerIntervalMs = getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG)
val logFlushOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushOffsetCheckpointIntervalMsProp).toLong val logFlushOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
val logFlushStartOffsetCheckpointIntervalMs = getInt(KafkaConfig.LogFlushStartOffsetCheckpointIntervalMsProp).toLong val logFlushStartOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
val logCleanupIntervalMs = getLong(KafkaConfig.LogCleanupIntervalMsProp) val logCleanupIntervalMs = getLong(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG)
def logCleanupPolicy = getList(KafkaConfig.LogCleanupPolicyProp) def logCleanupPolicy = getList(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG)
val offsetsRetentionMinutes = getInt(GroupCoordinatorConfig.OFFSETS_RETENTION_MINUTES_CONFIG) val offsetsRetentionMinutes = getInt(GroupCoordinatorConfig.OFFSETS_RETENTION_MINUTES_CONFIG)
val offsetsRetentionCheckIntervalMs = getLong(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG) val offsetsRetentionCheckIntervalMs = getLong(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG)
def logRetentionBytes = getLong(KafkaConfig.LogRetentionBytesProp) def logRetentionBytes = getLong(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG)
val logCleanerDedupeBufferSize = getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP) val logCleanerDedupeBufferSize = getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP)
val logCleanerDedupeBufferLoadFactor = getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP) val logCleanerDedupeBufferLoadFactor = getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP)
val logCleanerIoBufferSize = getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP) val logCleanerIoBufferSize = getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP)
@ -1260,40 +1142,40 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP) val logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP)
def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP) def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP)
val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP) val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP)
def logIndexSizeMaxBytes = getInt(KafkaConfig.LogIndexSizeMaxBytesProp) def logIndexSizeMaxBytes = getInt(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG)
def logIndexIntervalBytes = getInt(KafkaConfig.LogIndexIntervalBytesProp) def logIndexIntervalBytes = getInt(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG)
def logDeleteDelayMs = getLong(KafkaConfig.LogDeleteDelayMsProp) def logDeleteDelayMs = getLong(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG)
def logRollTimeMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeHoursProp)) def logRollTimeMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG))
def logRollTimeJitterMillis: java.lang.Long = Option(getLong(KafkaConfig.LogRollTimeJitterMillisProp)).getOrElse(60 * 60 * 1000L * getInt(KafkaConfig.LogRollTimeJitterHoursProp)) def logRollTimeJitterMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG))
def logFlushIntervalMs: java.lang.Long = Option(getLong(KafkaConfig.LogFlushIntervalMsProp)).getOrElse(getLong(KafkaConfig.LogFlushSchedulerIntervalMsProp)) def logFlushIntervalMs: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG)).getOrElse(getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG))
def minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp) def minInSyncReplicas = getInt(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG)
def logPreAllocateEnable: java.lang.Boolean = getBoolean(KafkaConfig.LogPreAllocateProp) def logPreAllocateEnable: java.lang.Boolean = getBoolean(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG)
// We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0` // We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
// is passed, `0.10.0-IV0` may be picked) // is passed, `0.10.0-IV0` may be picked)
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
private val logMessageFormatVersionString = getString(KafkaConfig.LogMessageFormatVersionProp) private val logMessageFormatVersionString = getString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG)
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
@deprecated("3.0") @deprecated("3.0")
lazy val logMessageFormatVersion = lazy val logMessageFormatVersion =
if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion)) if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion))
MetadataVersion.fromVersionString(LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION) MetadataVersion.fromVersionString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT)
else MetadataVersion.fromVersionString(logMessageFormatVersionString) else MetadataVersion.fromVersionString(logMessageFormatVersionString)
def logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp)) def logMessageTimestampType = TimestampType.forName(getString(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG))
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */ /* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
@deprecated("3.6") @deprecated("3.6")
def logMessageTimestampDifferenceMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp) def logMessageTimestampDifferenceMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG)
// In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility, // In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
// we are using its value if logMessageTimestampBeforeMaxMs default value hasn't changed. // we are using its value if logMessageTimestampBeforeMaxMs default value hasn't changed.
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
def logMessageTimestampBeforeMaxMs: Long = { def logMessageTimestampBeforeMaxMs: Long = {
val messageTimestampBeforeMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampBeforeMaxMsProp) val messageTimestampBeforeMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)
if (messageTimestampBeforeMaxMs != LogConfig.DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS) { if (messageTimestampBeforeMaxMs != ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT) {
messageTimestampBeforeMaxMs messageTimestampBeforeMaxMs
} else { } else {
logMessageTimestampDifferenceMaxMs logMessageTimestampDifferenceMaxMs
@ -1305,7 +1187,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
def logMessageTimestampAfterMaxMs: Long = { def logMessageTimestampAfterMaxMs: Long = {
val messageTimestampAfterMaxMs: Long = getLong(KafkaConfig.LogMessageTimestampAfterMaxMsProp) val messageTimestampAfterMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)
if (messageTimestampAfterMaxMs != Long.MaxValue) { if (messageTimestampAfterMaxMs != Long.MaxValue) {
messageTimestampAfterMaxMs messageTimestampAfterMaxMs
} else { } else {
@ -1313,7 +1195,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} }
} }
def logMessageDownConversionEnable: Boolean = getBoolean(KafkaConfig.LogMessageDownConversionEnableProp) def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
/** ********* Replication configuration ***********/ /** ********* Replication configuration ***********/
val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG) val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG)
@ -1518,10 +1400,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val millisInHour = 60L * millisInMinute val millisInHour = 60L * millisInMinute
val millis: java.lang.Long = val millis: java.lang.Long =
Option(getLong(KafkaConfig.LogRetentionTimeMillisProp)).getOrElse( Option(getLong(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)).getOrElse(
Option(getInt(KafkaConfig.LogRetentionTimeMinutesProp)) match { Option(getInt(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG)) match {
case Some(mins) => millisInMinute * mins case Some(mins) => millisInMinute * mins
case None => getInt(KafkaConfig.LogRetentionTimeHoursProp) * millisInHour case None => getInt(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG) * millisInHour
}) })
if (millis < 0) return -1 if (millis < 0) return -1
@ -1938,7 +1820,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
private def createBrokerWarningMessage: String = { private def createBrokerWarningMessage: String = {
s"Broker configuration ${KafkaConfig.LogMessageFormatVersionProp} with value $logMessageFormatVersionString is ignored " + s"Broker configuration ${ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG} with value $logMessageFormatVersionString is ignored " +
s"because the inter-broker protocol version `$interBrokerProtocolVersionString` is greater or equal than 3.0. " + 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." "This configuration is deprecated and it will be removed in Apache Kafka 4.0."
} }

View File

@ -123,7 +123,7 @@ object KafkaRaftServer {
/** /**
* Initialize the configured log directories, including both [[KafkaConfig.MetadataLogDirProp]] * Initialize the configured log directories, including both [[KafkaConfig.MetadataLogDirProp]]
* and [[KafkaConfig.LogDirProp]]. This method performs basic validation to ensure that all * and [[KafkaConfig.LOG_DIR_PROP]]. This method performs basic validation to ensure that all
* directories are accessible and have been initialized with consistent `meta.properties`. * directories are accessible and have been initialized with consistent `meta.properties`.
* *
* @param config The process configuration * @param config The process configuration

View File

@ -49,6 +49,8 @@ import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, S
import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.common.utils.Sanitizer
import org.apache.kafka.server.common.AdminOperationException import org.apache.kafka.server.common.AdminOperationException
import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals} import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals}
import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG
import org.apache.kafka.server.config.ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG
import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.storage.internals.log.LogConfig
import scala.collection.{Map, mutable, _} import scala.collection.{Map, mutable, _}
@ -79,10 +81,10 @@ class ZkAdminManager(val config: KafkaConfig,
private val configHelper = new ConfigHelper(metadataCache, config, new ZkConfigRepository(adminZkClient)) private val configHelper = new ConfigHelper(metadataCache, config, new ZkConfigRepository(adminZkClient))
private val createTopicPolicy = private val createTopicPolicy =
Option(config.getConfiguredInstance(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[CreateTopicPolicy])) Option(config.getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy]))
private val alterConfigPolicy = private val alterConfigPolicy =
Option(config.getConfiguredInstance(KafkaConfig.AlterConfigPolicyClassNameProp, classOf[AlterConfigPolicy])) Option(config.getConfiguredInstance(ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[AlterConfigPolicy]))
def hasDelayedTopicOperations = topicPurgatory.numDelayed != 0 def hasDelayedTopicOperations = topicPurgatory.numDelayed != 0

View File

@ -73,6 +73,7 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIRS_CONFIG;
import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG; import static org.apache.kafka.server.config.ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG;
@ -171,11 +172,11 @@ public class KafkaClusterTestKit implements AutoCloseable {
} }
if (brokerNode != null) { if (brokerNode != null) {
// Set the log.dirs according to the broker node setting (if there is a broker node) // Set the log.dirs according to the broker node setting (if there is a broker node)
props.put(KafkaConfig$.MODULE$.LogDirsProp(), props.put(LOG_DIRS_CONFIG,
String.join(",", brokerNode.logDataDirectories())); String.join(",", brokerNode.logDataDirectories()));
} else { } else {
// Set log.dirs equal to the metadata directory if there is just a controller. // Set log.dirs equal to the metadata directory if there is just a controller.
props.put(KafkaConfig$.MODULE$.LogDirsProp(), props.put(LOG_DIRS_CONFIG,
controllerNode.metadataDirectory()); controllerNode.metadataDirectory());
} }
props.put(KafkaConfig$.MODULE$.ListenerSecurityProtocolMapProp(), props.put(KafkaConfig$.MODULE$.ListenerSecurityProtocolMapProp(),

View File

@ -28,6 +28,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.requests.ListOffsetsResponse import org.apache.kafka.common.requests.ListOffsetsResponse
import org.apache.kafka.common.utils.{MockTime, Time, Utils} import org.apache.kafka.common.utils.{MockTime, Time, Utils}
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -345,8 +346,8 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
} }
// We use mock timer so the records can get removed if the test env is too busy to complete // We use mock timer so the records can get removed if the test env is too busy to complete
// tests before kafka-log-retention. Hence, we disable the retention to avoid failed tests // tests before kafka-log-retention. Hence, we disable the retention to avoid failed tests
props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "-1") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "-1")
props.setProperty(KafkaConfig.LogDirProp, dataFolder(index)) props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, dataFolder(index))
props props
}.map(KafkaConfig.fromProps) }.map(KafkaConfig.fromProps)
} }

View File

@ -24,6 +24,7 @@ import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.config.{ConfigException, ConfigResource, TopicConfig} import org.apache.kafka.common.config.{ConfigException, ConfigResource, TopicConfig}
import org.apache.kafka.common.errors.{InvalidConfigurationException, UnknownTopicOrPartitionException} import org.apache.kafka.common.errors.{InvalidConfigurationException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.utils.MockTime import org.apache.kafka.common.utils.MockTime
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentState} import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentState}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.function.Executable
@ -387,9 +388,9 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString) props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, storageManagerClassName) props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, storageManagerClassName)
props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, metadataManagerClassName) props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, metadataManagerClassName)
props.put(KafkaConfig.LogRetentionTimeMillisProp, "2000") props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "2000")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "1000") props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "1000")
props.put(KafkaConfig.LogRetentionBytesProp, "2048") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "2048")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "1024") props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "1024")
props props
} }

View File

@ -24,7 +24,7 @@ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp,
import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.config.KafkaSecurityConfigs import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs}
import org.apache.kafka.server.policy.AlterConfigPolicy import org.apache.kafka.server.policy.AlterConfigPolicy
import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.storage.internals.log.LogConfig
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue}
@ -76,7 +76,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
} }
private def overrideNodeConfigs(props: Properties): Unit = { private def overrideNodeConfigs(props: Properties): Unit = {
props.put(KafkaConfig.AlterConfigPolicyClassNameProp, classOf[Policy]) props.put(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[Policy])
} }
@ParameterizedTest @ParameterizedTest
@ -167,7 +167,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
assertEquals(4, configs.size) assertEquals(4, configs.size)
assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
assertEquals(LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value) assertEquals(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value)
assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
@ -199,7 +199,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
assertEquals(4, configs.size) assertEquals(4, configs.size)
assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
assertEquals(LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value) assertEquals(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT.toString, configs.get(topicResource1).get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value)
assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value) assertEquals("0.8", configs.get(topicResource2).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)

View File

@ -31,7 +31,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.config.KafkaSecurityConfigs import org.apache.kafka.server.config.KafkaSecurityConfigs
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout}
@ -198,9 +198,9 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
// verify that they show up in the "configs" output of CreateTopics. // verify that they show up in the "configs" output of CreateTopics.
if (testInfo.getTestMethod.toString.contains("testCreateTopicsReturnsConfigs")) { if (testInfo.getTestMethod.toString.contains("testCreateTopicsReturnsConfigs")) {
configs.foreach(config => { configs.foreach(config => {
config.setProperty(KafkaConfig.LogRollTimeHoursProp, "2") config.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "2")
config.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "240") config.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "240")
config.setProperty(KafkaConfig.LogRollTimeJitterMillisProp, "123") config.setProperty(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, "123")
}) })
} }
configs.foreach { config => configs.foreach { config =>

View File

@ -33,6 +33,7 @@ import org.apache.kafka.common.network.{ListenerName, Mode}
import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.{KafkaException, TopicPartition}
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -47,7 +48,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
def generateConfigs: scala.collection.Seq[KafkaConfig] = { def generateConfigs: scala.collection.Seq[KafkaConfig] = {
val overridingProps = new Properties() val overridingProps = new Properties()
val numServers = 2 val numServers = 2
overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 4.toString)
TestUtils.createBrokerConfigs( TestUtils.createBrokerConfigs(
numServers, numServers,
zkConnectOrNull, zkConnectOrNull,

View File

@ -26,7 +26,7 @@ import org.apache.kafka.common.message.FindCoordinatorRequestData
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinatorResponse} import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinatorResponse}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.ShutdownableThread import org.apache.kafka.server.util.ShutdownableThread
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, Disabled, Test} import org.junit.jupiter.api.{AfterEach, Disabled, Test}
@ -61,7 +61,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
properties.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") properties.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0")
properties.put(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG, maxGroupSize) properties.put(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG, maxGroupSize)
properties.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true") properties.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true")
properties.put(KafkaConfig.AutoCreateTopicsEnableProp, "false") properties.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
FixedPortTestUtils.createBrokerConfigs(brokerCount, zkConnect, enableControlledShutdown = false) FixedPortTestUtils.createBrokerConfigs(brokerCount, zkConnect, enableControlledShutdown = false)
.map(KafkaConfig.fromProps(_, properties)) .map(KafkaConfig.fromProps(_, properties))

View File

@ -27,6 +27,8 @@ import kafka.utils.{EmptyTestInfo, TestUtils}
import org.apache.kafka.clients.admin.NewTopic import org.apache.kafka.clients.admin.NewTopic
import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource} import org.junit.jupiter.params.provider.{Arguments, MethodSource}
@ -56,7 +58,7 @@ object ConsumerTopicCreationTest {
// configure server properties // configure server properties
this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown
this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, brokerAutoTopicCreationEnable.toString) this.serverConfig.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, brokerAutoTopicCreationEnable.toString)
// configure client properties // configure client properties
this.producerConfig.setProperty(ProducerConfig.CLIENT_ID_CONFIG, producerClientId) this.producerConfig.setProperty(ProducerConfig.CLIENT_ID_CONFIG, producerClientId)

View File

@ -40,7 +40,7 @@ import org.apache.kafka.common.security.auth._
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs} import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout}
@ -137,7 +137,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
// Some needed configuration for brokers, producers, and consumers // Some needed configuration for brokers, producers, and consumers
this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1")
this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3")
this.serverConfig.setProperty(KafkaConfig.MinInSyncReplicasProp, "3") this.serverConfig.setProperty(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3")
this.serverConfig.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "3") this.serverConfig.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "3")
this.serverConfig.setProperty(KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, "1500") this.serverConfig.setProperty(KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG, "1500")
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "group") this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "group")

View File

@ -18,11 +18,11 @@ package kafka.api
import java.util.Collections import java.util.Collections
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.api.{BeforeEach, TestInfo}
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -38,7 +38,7 @@ class LogAppendTimeTest extends IntegrationTestHarness {
val brokerCount: Int = 2 val brokerCount: Int = 2
// This will be used for the offsets topic as well // This will be used for the offsets topic as well
serverConfig.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.name) serverConfig.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.name)
serverConfig.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "2") serverConfig.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "2")
private val topic = "topic" private val topic = "topic"

View File

@ -13,7 +13,7 @@
package kafka.api package kafka.api
import java.util.{Locale, Properties} import java.util.{Locale, Properties}
import kafka.server.{KafkaConfig, KafkaServer} import kafka.server.KafkaServer
import kafka.utils.{JaasTestUtils, TestUtils} import kafka.utils.{JaasTestUtils, TestUtils}
import com.yammer.metrics.core.{Gauge, Histogram, Meter} import com.yammer.metrics.core.{Gauge, Histogram, Meter}
import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.consumer.Consumer
@ -25,7 +25,7 @@ import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.security.authenticator.TestJaasConfig import org.apache.kafka.common.security.authenticator.TestJaasConfig
import org.apache.kafka.server.config.ZkConfigs import org.apache.kafka.server.config.ZkConfigs
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteStorageMetrics} import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteStorageMetrics}
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
@ -46,7 +46,7 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {
private val kafkaServerJaasEntryName = private val kafkaServerJaasEntryName =
s"${listenerName.value.toLowerCase(Locale.ROOT)}.${JaasTestUtils.KafkaServerContextName}" s"${listenerName.value.toLowerCase(Locale.ROOT)}.${JaasTestUtils.KafkaServerContextName}"
this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "false") this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "false")
this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") this.serverConfig.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
this.serverConfig.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "2.8") this.serverConfig.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "2.8")
this.producerConfig.setProperty(ProducerConfig.LINGER_MS_CONFIG, "10") this.producerConfig.setProperty(ProducerConfig.LINGER_MS_CONFIG, "10")
// intentionally slow message down conversion via gzip compression to ensure we can measure the time it takes // intentionally slow message down conversion via gzip compression to ensure we can measure the time it takes

View File

@ -45,7 +45,7 @@ import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollectio
import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs} import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs, ZkConfigs}
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo}
@ -2222,7 +2222,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertEquals(compressionType, logConfig.originals.get(TopicConfig.COMPRESSION_TYPE_CONFIG)) assertEquals(compressionType, logConfig.originals.get(TopicConfig.COMPRESSION_TYPE_CONFIG))
assertNull(logConfig.originals.get(TopicConfig.RETENTION_BYTES_CONFIG)) assertNull(logConfig.originals.get(TopicConfig.RETENTION_BYTES_CONFIG))
assertEquals(LogConfig.DEFAULT_RETENTION_BYTES, logConfig.retentionSize) assertEquals(ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, logConfig.retentionSize)
} }
client = Admin.create(createConfig) client = Admin.create(createConfig)
@ -2528,7 +2528,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
client = Admin.create(super.createConfig) client = Admin.create(super.createConfig)
val newLogRetentionProperties = new Properties val newLogRetentionProperties = new Properties
newLogRetentionProperties.put(KafkaConfig.LogRetentionTimeMillisProp, "10800000") newLogRetentionProperties.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "10800000")
TestUtils.incrementalAlterConfigs(null, client, newLogRetentionProperties, perBrokerConfig = false) TestUtils.incrementalAlterConfigs(null, client, newLogRetentionProperties, perBrokerConfig = false)
.all().get(15, TimeUnit.SECONDS) .all().get(15, TimeUnit.SECONDS)
@ -2555,8 +2555,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
waitTimeMs = 60000L) waitTimeMs = 60000L)
waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault( waitUntilTrue(() => brokers.forall(_.config.originals.getOrDefault(
KafkaConfig.LogRetentionTimeMillisProp, "").toString.equals("10800000")), ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "").toString.equals("10800000")),
s"Timed out waiting for change to ${KafkaConfig.LogRetentionTimeMillisProp}", s"Timed out waiting for change to ${ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG}",
waitTimeMs = 60000L) waitTimeMs = 60000L)
val newTopics = Seq(new NewTopic("foo", Map((0: Integer) -> Seq[Integer](1, 2).asJava, val newTopics = Seq(new NewTopic("foo", Map((0: Integer) -> Seq[Integer](1, 2).asJava,

View File

@ -28,7 +28,7 @@ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch} import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -43,7 +43,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
val numServers = 2 val numServers = 2
val overridingProps = new Properties() val overridingProps = new Properties()
overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString)
overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, replicaFetchMaxPartitionBytes.toString) overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, replicaFetchMaxPartitionBytes.toString)
overridingProps.put(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_DOC, replicaFetchMaxResponseBytes.toString) overridingProps.put(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_DOC, replicaFetchMaxResponseBytes.toString)

View File

@ -31,7 +31,7 @@ import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.test.{TestUtils => JTestUtils} import org.apache.kafka.test.{TestUtils => JTestUtils}
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
@ -226,7 +226,7 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness {
private def serverProps(): Properties = { private def serverProps(): Properties = {
val serverProps = new Properties() val serverProps = new Properties()
serverProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) serverProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
// Set a smaller value for the number of partitions for the __consumer_offsets topic // Set a smaller value for the number of partitions for the __consumer_offsets topic
// so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long. // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long.
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)

View File

@ -16,12 +16,11 @@
*/ */
package kafka.api package kafka.api
import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.admin.NewPartitionReassignment import org.apache.kafka.clients.admin.NewPartitionReassignment
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
@ -36,7 +35,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
val producerCount: Int = 1 val producerCount: Int = 1
val brokerCount: Int = 2 val brokerCount: Int = 2
serverConfig.put(KafkaConfig.NumPartitionsProp, 2.toString) serverConfig.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, 2.toString)
serverConfig.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, 2.toString) serverConfig.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, 2.toString)
serverConfig.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverConfig.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString)

View File

@ -21,9 +21,9 @@ import java.util.Properties
import kafka.admin.{RackAwareMode, RackAwareTest} import kafka.admin.{RackAwareMode, RackAwareTest}
import kafka.integration.KafkaServerTestHarness import kafka.integration.KafkaServerTestHarness
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import org.apache.kafka.server.config.ReplicationConfigs
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import scala.collection.Map import scala.collection.Map
@ -33,7 +33,7 @@ class RackAwareAutoTopicCreationTest extends KafkaServerTestHarness with RackAwa
val numPartitions = 8 val numPartitions = 8
val replicationFactor = 2 val replicationFactor = 2
val overridingProps = new Properties() val overridingProps = new Properties()
overridingProps.put(KafkaConfig.NumPartitionsProp, numPartitions.toString) overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numPartitions.toString)
overridingProps.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, replicationFactor.toString) overridingProps.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, replicationFactor.toString)
def generateConfigs = def generateConfigs =

View File

@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.util.ShutdownableThread import org.apache.kafka.server.util.ShutdownableThread
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
@ -45,7 +45,7 @@ class TransactionsBounceTest extends IntegrationTestHarness {
private val inputTopic = "input-topic" private val inputTopic = "input-topic"
val overridingProps = new Properties() val overridingProps = new Properties()
overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString)
// Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic)
// so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long
@ -54,7 +54,7 @@ class TransactionsBounceTest extends IntegrationTestHarness {
overridingProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) overridingProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString)
overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 3.toString) overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 3.toString)
overridingProps.put(KafkaConfig.MinInSyncReplicasProp, 2.toString) overridingProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 2.toString)
overridingProps.put(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "10") // set small enough session timeout overridingProps.put(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "10") // set small enough session timeout
overridingProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") overridingProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0")
overridingProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString) overridingProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString)
@ -187,7 +187,7 @@ class TransactionsBounceTest extends IntegrationTestHarness {
private def createTopics() = { private def createTopics() = {
val topicConfig = new Properties() val topicConfig = new Properties()
topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 2.toString) topicConfig.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 2.toString)
createTopic(inputTopic, numPartitions, 3, topicConfig) createTopic(inputTopic, numPartitions, 3, topicConfig)
createTopic(outputTopic, numPartitions, 3, topicConfig) createTopic(outputTopic, numPartitions, 3, topicConfig)
} }

View File

@ -29,7 +29,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException}
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -201,7 +201,7 @@ class TransactionsExpirationTest extends KafkaServerTestHarness {
private def serverProps(): Properties = { private def serverProps(): Properties = {
val serverProps = new Properties() val serverProps = new Properties()
serverProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) serverProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
// Set a smaller value for the number of partitions for the __consumer_offsets topic // Set a smaller value for the number of partitions for the __consumer_offsets topic
// so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long. // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long.
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)

View File

@ -31,7 +31,7 @@ import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFe
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -61,7 +61,7 @@ class TransactionsTest extends IntegrationTestHarness {
def overridingProps(): Properties = { def overridingProps(): Properties = {
val props = new Properties() val props = new Properties()
props.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) props.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
// Set a smaller value for the number of partitions for the __consumer_offsets topic + // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long // Set a smaller value for the number of partitions for the __consumer_offsets topic + // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long
props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString)
@ -92,7 +92,7 @@ class TransactionsTest extends IntegrationTestHarness {
def topicConfig(): Properties = { def topicConfig(): Properties = {
val topicConfig = new Properties() val topicConfig = new Properties()
topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 2.toString) topicConfig.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 2.toString)
topicConfig topicConfig
} }

View File

@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -58,7 +58,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness {
override def setUp(testInfo: TestInfo): Unit = { override def setUp(testInfo: TestInfo): Unit = {
super.setUp(testInfo) super.setUp(testInfo)
val topicConfig = new Properties() val topicConfig = new Properties()
topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 1.toString) topicConfig.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, 1.toString)
createTopic(topic1, numPartitions, numBrokers, topicConfig) createTopic(topic1, numPartitions, numBrokers, topicConfig)
createTopic(topic2, numPartitions, numBrokers, topicConfig) createTopic(topic2, numPartitions, numBrokers, topicConfig)
@ -105,7 +105,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness {
private def serverProps() = { private def serverProps() = {
val serverProps = new Properties() val serverProps = new Properties()
serverProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) serverProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString)
serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 1.toString) serverProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 1.toString)
serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString)

View File

@ -62,7 +62,7 @@ import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializ
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ZkConfigs} import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ZkConfigs}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.record.BrokerCompressionType
import org.apache.kafka.server.util.ShutdownableThread import org.apache.kafka.server.util.ShutdownableThread
@ -135,11 +135,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
props.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "requested") props.put(KafkaSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "requested")
props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN") props.put(KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN")
props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(",")) props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, kafkaServerSaslMechanisms.mkString(","))
props.put(KafkaConfig.LogSegmentBytesProp, "2000") // low value to test log rolling on config update props.put(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "2000") // low value to test log rolling on config update
props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") // greater than one to test reducing threads props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") // greater than one to test reducing threads
props.put(KafkaConfig.PasswordEncoderSecretProp, "dynamic-config-secret") props.put(KafkaConfig.PasswordEncoderSecretProp, "dynamic-config-secret")
props.put(KafkaConfig.LogRetentionTimeMillisProp, 1680000000.toString) props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, 1680000000.toString)
props.put(KafkaConfig.LogRetentionTimeHoursProp, 168.toString) props.put(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, 168.toString)
props ++= sslProperties1 props ++= sslProperties1
props ++= securityProps(sslProperties1, KEYSTORE_PROPS, listenerPrefix(SecureInternal)) props ++= securityProps(sslProperties1, KEYSTORE_PROPS, listenerPrefix(SecureInternal))
@ -253,18 +253,18 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify a few log configs with and without synonyms // Verify a few log configs with and without synonyms
val expectedProps = new Properties val expectedProps = new Properties
expectedProps.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1680000000") expectedProps.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1680000000")
expectedProps.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "168") expectedProps.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "168")
expectedProps.setProperty(KafkaConfig.LogRollTimeHoursProp, "168") expectedProps.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "168")
expectedProps.setProperty(CleanerConfig.LOG_CLEANER_THREADS_PROP, "1") expectedProps.setProperty(CleanerConfig.LOG_CLEANER_THREADS_PROP, "1")
val logRetentionMs = configEntry(configDesc, KafkaConfig.LogRetentionTimeMillisProp) val logRetentionMs = configEntry(configDesc, ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)
verifyConfig(KafkaConfig.LogRetentionTimeMillisProp, logRetentionMs, verifyConfig(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, logRetentionMs,
isSensitive = false, isReadOnly = false, expectedProps) isSensitive = false, isReadOnly = false, expectedProps)
val logRetentionHours = configEntry(configDesc, KafkaConfig.LogRetentionTimeHoursProp) val logRetentionHours = configEntry(configDesc, ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG)
verifyConfig(KafkaConfig.LogRetentionTimeHoursProp, logRetentionHours, verifyConfig(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, logRetentionHours,
isSensitive = false, isReadOnly = true, expectedProps) isSensitive = false, isReadOnly = true, expectedProps)
val logRollHours = configEntry(configDesc, KafkaConfig.LogRollTimeHoursProp) val logRollHours = configEntry(configDesc, ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG)
verifyConfig(KafkaConfig.LogRollTimeHoursProp, logRollHours, verifyConfig(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, logRollHours,
isSensitive = false, isReadOnly = true, expectedProps) isSensitive = false, isReadOnly = true, expectedProps)
val logCleanerThreads = configEntry(configDesc, CleanerConfig.LOG_CLEANER_THREADS_PROP) val logCleanerThreads = configEntry(configDesc, CleanerConfig.LOG_CLEANER_THREADS_PROP)
verifyConfig(CleanerConfig.LOG_CLEANER_THREADS_PROP, logCleanerThreads, verifyConfig(CleanerConfig.LOG_CLEANER_THREADS_PROP, logCleanerThreads,
@ -272,14 +272,14 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
def synonymsList(configEntry: ConfigEntry): List[(String, ConfigSource)] = def synonymsList(configEntry: ConfigEntry): List[(String, ConfigSource)] =
configEntry.synonyms.asScala.map(s => (s.name, s.source)).toList configEntry.synonyms.asScala.map(s => (s.name, s.source)).toList
assertEquals(List((KafkaConfig.LogRetentionTimeMillisProp, ConfigSource.STATIC_BROKER_CONFIG), assertEquals(List((ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, ConfigSource.STATIC_BROKER_CONFIG),
(KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.STATIC_BROKER_CONFIG), (ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.STATIC_BROKER_CONFIG),
(KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.DEFAULT_CONFIG)), (ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.DEFAULT_CONFIG)),
synonymsList(logRetentionMs)) synonymsList(logRetentionMs))
assertEquals(List((KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.STATIC_BROKER_CONFIG), assertEquals(List((ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.STATIC_BROKER_CONFIG),
(KafkaConfig.LogRetentionTimeHoursProp, ConfigSource.DEFAULT_CONFIG)), (ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, ConfigSource.DEFAULT_CONFIG)),
synonymsList(logRetentionHours)) synonymsList(logRetentionHours))
assertEquals(List((KafkaConfig.LogRollTimeHoursProp, ConfigSource.DEFAULT_CONFIG)), synonymsList(logRollHours)) assertEquals(List((ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, ConfigSource.DEFAULT_CONFIG)), synonymsList(logRollHours))
assertEquals(List((CleanerConfig.LOG_CLEANER_THREADS_PROP, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads)) assertEquals(List((CleanerConfig.LOG_CLEANER_THREADS_PROP, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads))
} }
@ -581,7 +581,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
def testConsecutiveConfigChange(quorum: String): Unit = { def testConsecutiveConfigChange(quorum: String): Unit = {
val topic2 = "testtopic2" val topic2 = "testtopic2"
val topicProps = new Properties val topicProps = new Properties
topicProps.put(KafkaConfig.MinInSyncReplicasProp, "2") topicProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2")
TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, controllerServers, numPartitions = 1, replicationFactor = numServers, topicConfig = topicProps) TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, controllerServers, numPartitions = 1, replicationFactor = numServers, topicConfig = topicProps)
def getLogOrThrow(tp: TopicPartition): UnifiedLog = { def getLogOrThrow(tp: TopicPartition): UnifiedLog = {
@ -593,13 +593,13 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
} }
var log = getLogOrThrow(new TopicPartition(topic2, 0)) var log = getLogOrThrow(new TopicPartition(topic2, 0))
assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertTrue(log.config.overriddenConfigs.contains(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG))
assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) assertEquals("2", log.config.originals().get(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG).toString)
val props = new Properties val props = new Properties
props.put(KafkaConfig.MinInSyncReplicasProp, "3") props.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3")
// Make a broker-default config // Make a broker-default config
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.MinInSyncReplicasProp, "3")) reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "3"))
// Verify that all broker defaults have been updated again // Verify that all broker defaults have been updated again
servers.foreach { server => servers.foreach { server =>
props.forEach { (k, v) => props.forEach { (k, v) =>
@ -608,16 +608,16 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
} }
log = getLogOrThrow(new TopicPartition(topic2, 0)) log = getLogOrThrow(new TopicPartition(topic2, 0))
assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertTrue(log.config.overriddenConfigs.contains(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG))
assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config survives assertEquals("2", log.config.originals().get(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG).toString) // Verify topic-level config survives
// Make a second broker-default change // Make a second broker-default change
props.clear() props.clear()
props.put(KafkaConfig.LogRetentionTimeMillisProp, "604800000") props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "604800000")
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogRetentionTimeMillisProp, "604800000")) reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "604800000"))
log = getLogOrThrow(new TopicPartition(topic2, 0)) log = getLogOrThrow(new TopicPartition(topic2, 0))
assertTrue(log.config.overriddenConfigs.contains(KafkaConfig.MinInSyncReplicasProp)) assertTrue(log.config.overriddenConfigs.contains(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG))
assertEquals("2", log.config.originals().get(KafkaConfig.MinInSyncReplicasProp).toString) // Verify topic-level config still survives assertEquals("2", log.config.originals().get(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG).toString) // Verify topic-level config still survives
} }
@Test @Test
@ -627,31 +627,31 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val (producerThread, consumerThread) = startProduceConsume(retries = 0) val (producerThread, consumerThread) = startProduceConsume(retries = 0)
val props = new Properties val props = new Properties
props.put(KafkaConfig.LogSegmentBytesProp, "4000") props.put(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "4000")
props.put(KafkaConfig.LogRollTimeMillisProp, TimeUnit.HOURS.toMillis(2).toString) props.put(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, TimeUnit.HOURS.toMillis(2).toString)
props.put(KafkaConfig.LogRollTimeJitterMillisProp, TimeUnit.HOURS.toMillis(1).toString) props.put(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG, TimeUnit.HOURS.toMillis(1).toString)
props.put(KafkaConfig.LogIndexSizeMaxBytesProp, "100000") props.put(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, "100000")
props.put(KafkaConfig.LogFlushIntervalMessagesProp, "1000") props.put(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG, "1000")
props.put(KafkaConfig.LogFlushIntervalMsProp, "60000") props.put(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, "60000")
props.put(KafkaConfig.LogRetentionBytesProp, "10000000") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "10000000")
props.put(KafkaConfig.LogRetentionTimeMillisProp, TimeUnit.DAYS.toMillis(1).toString) props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, TimeUnit.DAYS.toMillis(1).toString)
props.put(KafkaConfig.MessageMaxBytesProp, "100000") props.put(KafkaConfig.MessageMaxBytesProp, "100000")
props.put(KafkaConfig.LogIndexIntervalBytesProp, "10000") props.put(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG, "10000")
props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString) props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString)
props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000") props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000")
props.put(KafkaConfig.LogDeleteDelayMsProp, "60000") props.put(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "60000")
props.put(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP, "0.3") props.put(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP, "0.3")
props.put(KafkaConfig.LogCleanupPolicyProp, "delete") props.put(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG, "delete")
props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false") props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false")
props.put(KafkaConfig.MinInSyncReplicasProp, "2") props.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2")
props.put(KafkaConfig.CompressionTypeProp, "gzip") props.put(KafkaConfig.CompressionTypeProp, "gzip")
props.put(KafkaConfig.LogPreAllocateProp, true.toString) props.put(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG, true.toString)
props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.toString)
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000")
props.put(KafkaConfig.LogMessageTimestampBeforeMaxMsProp, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, "1000")
props.put(KafkaConfig.LogMessageTimestampAfterMaxMsProp, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, "1000")
props.put(KafkaConfig.LogMessageDownConversionEnableProp, "false") props.put(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "false")
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogSegmentBytesProp, "4000")) reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "4000"))
// Verify that all broker defaults have been updated // Verify that all broker defaults have been updated
servers.foreach { server => servers.foreach { server =>
@ -669,7 +669,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
TestUtils.waitUntilTrue(() => log.config.segmentSize == 4000, "Existing topic config using defaults not updated") TestUtils.waitUntilTrue(() => log.config.segmentSize == 4000, "Existing topic config using defaults not updated")
props.asScala.foreach { case (k, v) => props.asScala.foreach { case (k, v) =>
val logConfigName = DynamicLogConfig.KafkaConfigToLogConfigName(k) val logConfigName = DynamicLogConfig.KafkaConfigToLogConfigName(k)
val expectedValue = if (k == KafkaConfig.LogCleanupPolicyProp) s"[$v]" else v val expectedValue = if (k == ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG) s"[$v]" else v
assertEquals(expectedValue, log.config.originals.get(logConfigName).toString, assertEquals(expectedValue, log.config.originals.get(logConfigName).toString,
s"Not reconfigured $logConfigName for existing log") s"Not reconfigured $logConfigName for existing log")
} }
@ -686,19 +686,19 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify that we can alter subset of log configs // Verify that we can alter subset of log configs
props.clear() props.clear()
props.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.CREATE_TIME.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.CREATE_TIME.toString)
props.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000")
props.put(KafkaConfig.LogMessageTimestampBeforeMaxMsProp, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, "1000")
props.put(KafkaConfig.LogMessageTimestampAfterMaxMsProp, "1000") props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, "1000")
reconfigureServers(props, perBrokerConfig = false, (KafkaConfig.LogMessageTimestampTypeProp, TimestampType.CREATE_TIME.toString)) reconfigureServers(props, perBrokerConfig = false, (ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.CREATE_TIME.toString))
consumerThread.waitForMatchingRecords(record => record.timestampType == TimestampType.CREATE_TIME) consumerThread.waitForMatchingRecords(record => record.timestampType == TimestampType.CREATE_TIME)
// Verify that invalid configs are not applied // Verify that invalid configs are not applied
val invalidProps = Map( val invalidProps = Map(
KafkaConfig.LogMessageTimestampDifferenceMaxMsProp -> "abc", // Invalid type ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG -> "abc", // Invalid type
KafkaConfig.LogMessageTimestampBeforeMaxMsProp -> "abc", // Invalid type ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG -> "abc", // Invalid type
KafkaConfig.LogMessageTimestampAfterMaxMsProp -> "abc", // Invalid type ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG -> "abc", // Invalid type
KafkaConfig.LogMessageTimestampTypeProp -> "invalid", // Invalid value ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG -> "invalid", // Invalid value
KafkaConfig.LogRollTimeMillisProp -> "0" // Fails KafkaConfig validation ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG -> "0" // Fails KafkaConfig validation
) )
invalidProps.foreach { case (k, v) => invalidProps.foreach { case (k, v) =>
val newProps = new Properties val newProps = new Properties
@ -710,14 +710,14 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify that even though broker defaults can be defined at default cluster level for consistent // Verify that even though broker defaults can be defined at default cluster level for consistent
// configuration across brokers, they can also be defined at per-broker level for testing // configuration across brokers, they can also be defined at per-broker level for testing
props.clear() props.clear()
props.put(KafkaConfig.LogIndexSizeMaxBytesProp, "500000") props.put(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, "500000")
props.put(KafkaConfig.LogRetentionTimeMillisProp, TimeUnit.DAYS.toMillis(2).toString) props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, TimeUnit.DAYS.toMillis(2).toString)
alterConfigsOnServer(servers.head, props) alterConfigsOnServer(servers.head, props)
assertEquals(500000, servers.head.config.values.get(KafkaConfig.LogIndexSizeMaxBytesProp)) assertEquals(500000, servers.head.config.values.get(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG))
assertEquals(TimeUnit.DAYS.toMillis(2), servers.head.config.values.get(KafkaConfig.LogRetentionTimeMillisProp)) assertEquals(TimeUnit.DAYS.toMillis(2), servers.head.config.values.get(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG))
servers.tail.foreach { server => servers.tail.foreach { server =>
assertEquals(LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, server.config.values.get(KafkaConfig.LogIndexSizeMaxBytesProp)) assertEquals(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT, server.config.values.get(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG))
assertEquals(1680000000L, server.config.values.get(KafkaConfig.LogRetentionTimeMillisProp)) assertEquals(1680000000L, server.config.values.get(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG))
} }
// Verify that produce/consume worked throughout this test without any retries in producer // Verify that produce/consume worked throughout this test without any retries in producer
@ -726,17 +726,17 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
// Verify that configuration at both per-broker level and default cluster level could be deleted and // Verify that configuration at both per-broker level and default cluster level could be deleted and
// the default value should be restored // the default value should be restored
props.clear() props.clear()
props.put(KafkaConfig.LogRetentionTimeMillisProp, "") props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "")
props.put(KafkaConfig.LogIndexSizeMaxBytesProp, "") props.put(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG, "")
TestUtils.incrementalAlterConfigs(servers.take(1), adminClients.head, props, perBrokerConfig = true, opType = OpType.DELETE).all.get TestUtils.incrementalAlterConfigs(servers.take(1), adminClients.head, props, perBrokerConfig = true, opType = OpType.DELETE).all.get
TestUtils.incrementalAlterConfigs(servers, adminClients.head, props, perBrokerConfig = false, opType = OpType.DELETE).all.get TestUtils.incrementalAlterConfigs(servers, adminClients.head, props, perBrokerConfig = false, opType = OpType.DELETE).all.get
servers.foreach { server => servers.foreach { server =>
waitForConfigOnServer(server, KafkaConfig.LogRetentionTimeMillisProp, 1680000000.toString) waitForConfigOnServer(server, ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, 1680000000.toString)
} }
servers.foreach { server => servers.foreach { server =>
val log = server.logManager.getLog(new TopicPartition(topic, 0)).getOrElse(throw new IllegalStateException("Log not found")) 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 // Verify default values for these two configurations are restored on all brokers
TestUtils.waitUntilTrue(() => log.config.maxIndexSize == LogConfig.DEFAULT_SEGMENT_INDEX_BYTES && log.config.retentionMs == 1680000000L, TestUtils.waitUntilTrue(() => log.config.maxIndexSize == ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT && log.config.retentionMs == 1680000000L,
"Existing topic config using defaults not updated") "Existing topic config using defaults not updated")
} }
} }
@ -876,7 +876,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
fetcherThreadPrefix, mayReceiveDuplicates = false) fetcherThreadPrefix, mayReceiveDuplicates = false)
verifyThreadPoolResize(KafkaConfig.BackgroundThreadsProp, config.backgroundThreads, verifyThreadPoolResize(KafkaConfig.BackgroundThreadsProp, config.backgroundThreads,
"kafka-scheduler-", mayReceiveDuplicates = false) "kafka-scheduler-", mayReceiveDuplicates = false)
verifyThreadPoolResize(KafkaConfig.NumRecoveryThreadsPerDataDirProp, config.numRecoveryThreadsPerDataDir, verifyThreadPoolResize(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, config.numRecoveryThreadsPerDataDir,
"", mayReceiveDuplicates = false) "", mayReceiveDuplicates = false)
verifyThreadPoolResize(KafkaConfig.NumNetworkThreadsProp, config.numNetworkThreads, verifyThreadPoolResize(KafkaConfig.NumNetworkThreadsProp, config.numNetworkThreads,
networkThreadPrefix, mayReceiveDuplicates = true) networkThreadPrefix, mayReceiveDuplicates = true)

View File

@ -25,6 +25,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.FetchResponse import org.apache.kafka.common.requests.FetchResponse
import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.serialization.ByteArrayDeserializer
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{Disabled, Timeout} import org.junit.jupiter.api.{Disabled, Timeout}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -45,7 +46,7 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
def overridingProps: Properties = { def overridingProps: Properties = {
val props = new Properties val props = new Properties
props.put(KafkaConfig.NumPartitionsProp, numParts.toString) props.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numParts.toString)
props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, numNodes.toString) props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, numNodes.toString)
props props
} }

View File

@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.raft._ import org.apache.kafka.raft._
import org.apache.kafka.raft.internals.BatchBuilder import org.apache.kafka.raft.internals.BatchBuilder
import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.common.serialization.RecordSerde
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason} import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason}
@ -842,7 +843,7 @@ final class KafkaMetadataLogTest {
retentionMillis = 60 * 1000, retentionMillis = 60 * 1000,
maxBatchSizeInBytes = 512, maxBatchSizeInBytes = 512,
maxFetchSizeInBytes = DefaultMetadataLogConfig.maxFetchSizeInBytes, maxFetchSizeInBytes = DefaultMetadataLogConfig.maxFetchSizeInBytes,
fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
nodeId = 1 nodeId = 1
) )
config.copy() config.copy()
@ -1024,7 +1025,7 @@ object KafkaMetadataLogTest {
retentionMillis = 60 * 1000, retentionMillis = 60 * 1000,
maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES, maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES,
maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES, maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES,
fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
nodeId = 1 nodeId = 1
) )

View File

@ -26,6 +26,7 @@ import kafka.server.KafkaConfig
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.log4j.Logger import org.apache.log4j.Logger
import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.api.{AfterEach, Test}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -38,7 +39,7 @@ class ControllerFailoverTest extends KafkaServerTestHarness with Logging {
val topic = "topic1" val topic = "topic1"
val overridingProps = new Properties() val overridingProps = new Properties()
val metrics = new Metrics() val metrics = new Metrics()
overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numParts.toString)
override def generateConfigs = TestUtils.createBrokerConfigs(numNodes, zkConnect) override def generateConfigs = TestUtils.createBrokerConfigs(numNodes, zkConnect)
.map(KafkaConfig.fromProps(_, overridingProps)) .map(KafkaConfig.fromProps(_, overridingProps))

View File

@ -24,7 +24,7 @@ import kafka.utils.{Logging, TestUtils}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.api.{BeforeEach, TestInfo}
import com.yammer.metrics.core.Gauge import com.yammer.metrics.core.Gauge
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
@ -40,7 +40,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with
private val overridingProps = new Properties private val overridingProps = new Properties
overridingProps.put(KafkaConfig.DeleteTopicEnableProp, "true") overridingProps.put(KafkaConfig.DeleteTopicEnableProp, "true")
overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, "false") overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
// speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation // speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation
// But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment // But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment
overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, "4000") overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, "4000")

View File

@ -22,12 +22,13 @@ import scala.collection.Seq
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
class MinIsrConfigTest extends KafkaServerTestHarness { class MinIsrConfigTest extends KafkaServerTestHarness {
val overridingProps = new Properties() val overridingProps = new Properties()
overridingProps.put(KafkaConfig.MinInSyncReplicasProp, "5") overridingProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "5")
def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps(_, overridingProps)) def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps(_, overridingProps))
@ParameterizedTest @ParameterizedTest

View File

@ -27,7 +27,7 @@ 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.common.MetadataVersion.IBP_3_0_IV1
import org.apache.kafka.server.config.KafkaSecurityConfigs import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator} import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -48,7 +48,7 @@ class LogConfigTest {
@Test @Test
def ensureNoStaticInitializationOrderDependency(): Unit = { def ensureNoStaticInitializationOrderDependency(): Unit = {
// Access any KafkaConfig val to load KafkaConfig object before LogConfig. // Access any KafkaConfig val to load KafkaConfig object before LogConfig.
assertNotNull(KafkaConfig.LogRetentionTimeMillisProp) assertNotNull(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)
assertTrue(LogConfig.configNames.asScala assertTrue(LogConfig.configNames.asScala
.filter(config => !LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS.contains(config)) .filter(config => !LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS.contains(config))
.forall { config => .forall { config =>
@ -64,10 +64,10 @@ class LogConfigTest {
val millisInDay = 24L * millisInHour val millisInDay = 24L * millisInHour
val bytesInGB: Long = 1024 * 1024 * 1024 val bytesInGB: Long = 1024 * 1024 * 1024
val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
kafkaProps.put(KafkaConfig.LogRollTimeHoursProp, "2") kafkaProps.put(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "2")
kafkaProps.put(KafkaConfig.LogRollTimeJitterHoursProp, "2") kafkaProps.put(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG, "2")
kafkaProps.put(KafkaConfig.LogRetentionTimeHoursProp, "960") // 40 days kafkaProps.put(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "960") // 40 days
kafkaProps.put(KafkaConfig.LogMessageFormatVersionProp, "0.11.0") kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "2592000000") // 30 days kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "2592000000") // 30 days
kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "4294967296") // 4 GB kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "4294967296") // 4 GB
@ -173,7 +173,7 @@ class LogConfigTest {
val deleteDelayKey = configDef.configKeys.get(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG) val deleteDelayKey = configDef.configKeys.get(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG)
val deleteDelayServerDefault = configDef.getConfigValue(deleteDelayKey, LogConfig.SERVER_DEFAULT_HEADER_NAME) val deleteDelayServerDefault = configDef.getConfigValue(deleteDelayKey, LogConfig.SERVER_DEFAULT_HEADER_NAME)
assertEquals(KafkaConfig.LogDeleteDelayMsProp, deleteDelayServerDefault) assertEquals(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, deleteDelayServerDefault)
val keyWithNoServerMapping = configDef.configKeys.get(configNameWithNoServerMapping) val keyWithNoServerMapping = configDef.configKeys.get(configNameWithNoServerMapping)
val nullServerDefault = configDef.getConfigValue(keyWithNoServerMapping, LogConfig.SERVER_DEFAULT_HEADER_NAME) val nullServerDefault = configDef.getConfigValue(keyWithNoServerMapping, LogConfig.SERVER_DEFAULT_HEADER_NAME)
@ -184,8 +184,8 @@ class LogConfigTest {
def testOverriddenConfigsAsLoggableString(): Unit = { def testOverriddenConfigsAsLoggableString(): Unit = {
val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
kafkaProps.put("unknown.broker.password.config", "aaaaa") kafkaProps.put("unknown.broker.password.config", "aaaaa")
kafkaProps.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "50")
kafkaProps.put(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, "somekeypassword") kafkaProps.put(KafkaSecurityConfigs.SSL_KEY_PASSWORD_CONFIG, "somekeypassword")
kafkaProps.put(KafkaConfig.LogRetentionBytesProp, "50")
val kafkaConfig = KafkaConfig.fromProps(kafkaProps) val kafkaConfig = KafkaConfig.fromProps(kafkaProps)
val topicOverrides = new Properties val topicOverrides = new Properties
// Only set as a topic config // Only set as a topic config
@ -238,7 +238,7 @@ class LogConfigTest {
// Local retention defaults are derived from retention properties which can be default or custom. // Local retention defaults are derived from retention properties which can be default or custom.
assertEquals(LogConfig.DEFAULT_RETENTION_MS, logConfig.localRetentionMs) assertEquals(LogConfig.DEFAULT_RETENTION_MS, logConfig.localRetentionMs)
assertEquals(LogConfig.DEFAULT_RETENTION_BYTES, logConfig.localRetentionBytes) assertEquals(ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, logConfig.localRetentionBytes)
} }
@Test @Test
@ -342,7 +342,7 @@ class LogConfigTest {
def testTopicCreationWithInvalidRetentionTime(sysRemoteStorageEnabled: Boolean): Unit = { def testTopicCreationWithInvalidRetentionTime(sysRemoteStorageEnabled: Boolean): Unit = {
val kafkaProps = TestUtils.createDummyBrokerConfig() val kafkaProps = TestUtils.createDummyBrokerConfig()
kafkaProps.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString) kafkaProps.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
kafkaProps.put(KafkaConfig.LogRetentionTimeMillisProp, "1000") kafkaProps.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1000")
kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "900") kafkaProps.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "900")
val kafkaConfig = KafkaConfig.fromProps(kafkaProps) val kafkaConfig = KafkaConfig.fromProps(kafkaProps)
@ -364,7 +364,7 @@ class LogConfigTest {
def testTopicCreationWithInvalidRetentionSize(sysRemoteStorageEnabled: Boolean): Unit = { def testTopicCreationWithInvalidRetentionSize(sysRemoteStorageEnabled: Boolean): Unit = {
val props = TestUtils.createDummyBrokerConfig() val props = TestUtils.createDummyBrokerConfig()
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString) props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
props.put(KafkaConfig.LogRetentionBytesProp, "1024") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "1024")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "512") props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "512")
val kafkaConfig = KafkaConfig.fromProps(props) val kafkaConfig = KafkaConfig.fromProps(props)
@ -386,7 +386,7 @@ class LogConfigTest {
def testValidateBrokerLogConfigs(sysRemoteStorageEnabled: Boolean): Unit = { def testValidateBrokerLogConfigs(sysRemoteStorageEnabled: Boolean): Unit = {
val props = TestUtils.createDummyBrokerConfig() val props = TestUtils.createDummyBrokerConfig()
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString) props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, sysRemoteStorageEnabled.toString)
props.put(KafkaConfig.LogRetentionBytesProp, "1024") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "1024")
props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "2048") props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "2048")
val kafkaConfig = KafkaConfig.fromProps(props) val kafkaConfig = KafkaConfig.fromProps(props)
@ -399,8 +399,8 @@ class LogConfigTest {
} }
} }
/* Verify that when the deprecated config LogMessageTimestampDifferenceMaxMsProp has non default value the new configs /* Verify that when the deprecated config LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG has non default value the new configs
* LogMessageTimestampBeforeMaxMsProp and LogMessageTimestampAfterMaxMsProp are not changed from the default we are using * LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG and LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG are not changed from the default we are using
* the deprecated config for backward compatibility. * the deprecated config for backward compatibility.
* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details */ * See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details */
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
@ -408,9 +408,9 @@ class LogConfigTest {
def testTimestampBeforeMaxMsUsesDeprecatedConfig(): Unit = { def testTimestampBeforeMaxMsUsesDeprecatedConfig(): Unit = {
val oneDayInMillis = 24 * 60 * 60 * 1000L val oneDayInMillis = 24 * 60 * 60 * 1000L
val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") val kafkaProps = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
kafkaProps.put(KafkaConfig.LogMessageTimestampBeforeMaxMsProp, Long.MaxValue.toString) kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.MaxValue.toString)
kafkaProps.put(KafkaConfig.LogMessageTimestampAfterMaxMsProp, Long.MaxValue.toString) kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.MaxValue.toString)
kafkaProps.put(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp, oneDayInMillis.toString) kafkaProps.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, oneDayInMillis.toString)
val logProps = KafkaConfig.fromProps(kafkaProps).extractLogConfigMap val logProps = KafkaConfig.fromProps(kafkaProps).extractLogConfigMap
assertEquals(oneDayInMillis, logProps.get(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)) assertEquals(oneDayInMillis, logProps.get(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG))

View File

@ -32,6 +32,7 @@ import java.nio.file.Files
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.Scheduler import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile
import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, FetchDataInfo, FetchIsolation, LazyIndex, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetsListener, LogSegment, ProducerStateManager, ProducerStateManagerConfig, TransactionIndex} import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, FetchDataInfo, FetchIsolation, LazyIndex, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetsListener, LogSegment, ProducerStateManager, ProducerStateManagerConfig, TransactionIndex}
@ -58,14 +59,14 @@ object LogTestUtils {
segmentBytes: Int = LogConfig.DEFAULT_SEGMENT_BYTES, segmentBytes: Int = LogConfig.DEFAULT_SEGMENT_BYTES,
retentionMs: Long = LogConfig.DEFAULT_RETENTION_MS, retentionMs: Long = LogConfig.DEFAULT_RETENTION_MS,
localRetentionMs: Long = LogConfig.DEFAULT_LOCAL_RETENTION_MS, localRetentionMs: Long = LogConfig.DEFAULT_LOCAL_RETENTION_MS,
retentionBytes: Long = LogConfig.DEFAULT_RETENTION_BYTES, retentionBytes: Long = ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT,
localRetentionBytes: Long = LogConfig.DEFAULT_LOCAL_RETENTION_BYTES, localRetentionBytes: Long = LogConfig.DEFAULT_LOCAL_RETENTION_BYTES,
segmentJitterMs: Long = LogConfig.DEFAULT_SEGMENT_JITTER_MS, segmentJitterMs: Long = LogConfig.DEFAULT_SEGMENT_JITTER_MS,
cleanupPolicy: String = LogConfig.DEFAULT_CLEANUP_POLICY, cleanupPolicy: String = ServerLogConfigs.LOG_CLEANUP_POLICY_DEFAULT,
maxMessageBytes: Int = LogConfig.DEFAULT_MAX_MESSAGE_BYTES, maxMessageBytes: Int = LogConfig.DEFAULT_MAX_MESSAGE_BYTES,
indexIntervalBytes: Int = LogConfig.DEFAULT_INDEX_INTERVAL_BYTES, indexIntervalBytes: Int = ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT,
segmentIndexBytes: Int = LogConfig.DEFAULT_SEGMENT_INDEX_BYTES, segmentIndexBytes: Int = ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT,
fileDeleteDelayMs: Long = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, fileDeleteDelayMs: Long = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
remoteLogStorageEnable: Boolean = LogConfig.DEFAULT_REMOTE_STORAGE_ENABLE): LogConfig = { remoteLogStorageEnable: Boolean = LogConfig.DEFAULT_REMOTE_STORAGE_ENABLE): LogConfig = {
val logProps = new Properties() val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long)

View File

@ -33,6 +33,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.metrics.JmxReporter import org.apache.kafka.common.metrics.JmxReporter
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.migration.ZkMigrationState import org.apache.kafka.metadata.migration.ZkMigrationState
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.Timeout import org.junit.jupiter.api.Timeout
@ -46,7 +47,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
val requiredKafkaServerPrefix = "kafka.server:type=KafkaServer,name" val requiredKafkaServerPrefix = "kafka.server:type=KafkaServer,name"
val overridingProps = new Properties val overridingProps = new Properties
overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) overridingProps.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numParts.toString)
overridingProps.put(JmxReporter.EXCLUDE_CONFIG, s"$requiredKafkaServerPrefix=ClusterId") overridingProps.put(JmxReporter.EXCLUDE_CONFIG, s"$requiredKafkaServerPrefix=ClusterId")
def generateConfigs: Seq[KafkaConfig] = def generateConfigs: Seq[KafkaConfig] =

View File

@ -30,7 +30,7 @@ import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.config.ZkConfigs import org.apache.kafka.server.config.ZkConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -50,7 +50,7 @@ class RaftManagerTest {
): KafkaConfig = { ): KafkaConfig = {
val props = new Properties val props = new Properties
logDir.foreach { value => logDir.foreach { value =>
props.setProperty(KafkaConfig.LogDirProp, value.toString) props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, value.toString)
} }
if (migrationEnabled) { if (migrationEnabled) {
metadataDir.foreach { value => metadataDir.foreach { value =>
@ -74,7 +74,7 @@ class RaftManagerTest {
): KafkaConfig = { ): KafkaConfig = {
val props = new Properties val props = new Properties
logDir.foreach { value => logDir.foreach { value =>
props.setProperty(KafkaConfig.LogDirProp, value.toString) props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, value.toString)
} }
metadataDir.foreach { value => metadataDir.foreach { value =>
props.setProperty(KafkaConfig.MetadataLogDirProp, value.toString) props.setProperty(KafkaConfig.MetadataLogDirProp, value.toString)

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData
import org.apache.kafka.common.message.CreateTopicsRequestData._ import org.apache.kafka.common.message.CreateTopicsRequestData._
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
@ -33,7 +34,7 @@ import scala.jdk.CollectionConverters._
abstract class AbstractCreateTopicsRequestTest extends BaseRequestTest { abstract class AbstractCreateTopicsRequestTest extends BaseRequestTest {
override def brokerPropertyOverrides(properties: Properties): Unit = override def brokerPropertyOverrides(properties: Properties): Unit =
properties.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) properties.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
def topicsReq(topics: Seq[CreatableTopic], def topicsReq(topics: Seq[CreatableTopic],
timeout: Integer = 10000, timeout: Integer = 10000,

View File

@ -30,6 +30,7 @@ import org.apache.kafka.common.message.{FindCoordinatorRequestData, InitProducer
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType
import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, FindCoordinatorRequest, FindCoordinatorResponse, InitProducerIdRequest, InitProducerIdResponse} import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, FindCoordinatorRequest, FindCoordinatorResponse, InitProducerIdRequest, InitProducerIdResponse}
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -43,7 +44,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
val numPartitions = 1 val numPartitions = 1
override def brokerPropertyOverrides(properties: Properties): Unit = { override def brokerPropertyOverrides(properties: Properties): Unit = {
properties.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) properties.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString)
} }
@BeforeEach @BeforeEach

View File

@ -25,6 +25,7 @@ import org.apache.kafka.common.message.{BrokerHeartbeatResponseData, BrokerRegis
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
import org.apache.kafka.metadata.BrokerState import org.apache.kafka.metadata.BrokerState
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
@ -35,7 +36,7 @@ import scala.jdk.CollectionConverters._
class BrokerLifecycleManagerTest { class BrokerLifecycleManagerTest {
def configProperties = { def configProperties = {
val properties = new Properties() val properties = new Properties()
properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo") properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "broker") properties.setProperty(KafkaConfig.ProcessRolesProp, "broker")
properties.setProperty(KafkaConfig.NodeIdProp, "1") properties.setProperty(KafkaConfig.NodeIdProp, "1")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9093") properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9093")

View File

@ -28,6 +28,7 @@ import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.metadata.{ListenerInfo, RecordTestUtils, VersionRange} import org.apache.kafka.metadata.{ListenerInfo, RecordTestUtils, VersionRange}
import org.apache.kafka.raft.LeaderAndEpoch import org.apache.kafka.raft.LeaderAndEpoch
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.test.TestUtils import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
@ -45,7 +46,7 @@ class ControllerRegistrationManagerTest {
private def configProperties = { private def configProperties = {
val properties = new Properties() val properties = new Properties()
properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo") properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller") properties.setProperty(KafkaConfig.ProcessRolesProp, "controller")
properties.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, s"CONTROLLER:PLAINTEXT") properties.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, s"CONTROLLER:PLAINTEXT")
properties.setProperty(KafkaConfig.ListenersProp, s"CONTROLLER://localhost:8001") properties.setProperty(KafkaConfig.ListenersProp, s"CONTROLLER://localhost:8001")

View File

@ -23,6 +23,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.PolicyViolationException import org.apache.kafka.common.errors.PolicyViolationException
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG
import org.apache.kafka.server.policy.CreateTopicPolicy import org.apache.kafka.server.policy.CreateTopicPolicy
import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -35,12 +36,12 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest
override def brokerPropertyOverrides(properties: Properties): Unit = { override def brokerPropertyOverrides(properties: Properties): Unit = {
super.brokerPropertyOverrides(properties) super.brokerPropertyOverrides(properties)
properties.put(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[Policy].getName) properties.put(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[Policy].getName)
} }
override def kraftControllerConfigs(): Seq[Properties] = { override def kraftControllerConfigs(): Seq[Properties] = {
val properties = new Properties() val properties = new Properties()
properties.put(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[Policy].getName) properties.put(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[Policy].getName)
Seq(properties) Seq(properties)
} }

View File

@ -38,7 +38,7 @@ import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs, ZkConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.server.util.KafkaScheduler
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig}
import org.apache.kafka.test.MockMetricsReporter import org.apache.kafka.test.MockMetricsReporter
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -135,7 +135,7 @@ class DynamicBrokerConfigTest {
origProps.put(KafkaConfig.NumIoThreadsProp, "4") origProps.put(KafkaConfig.NumIoThreadsProp, "4")
origProps.put(KafkaConfig.NumNetworkThreadsProp, "2") origProps.put(KafkaConfig.NumNetworkThreadsProp, "2")
origProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") origProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1")
origProps.put(KafkaConfig.NumRecoveryThreadsPerDataDirProp, "1") origProps.put(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, "1")
origProps.put(KafkaConfig.BackgroundThreadsProp, "3") origProps.put(KafkaConfig.BackgroundThreadsProp, "3")
val config = KafkaConfig(origProps) val config = KafkaConfig(origProps)
@ -181,7 +181,7 @@ class DynamicBrokerConfigTest {
assertEquals(2, config.numReplicaFetchers) assertEquals(2, config.numReplicaFetchers)
Mockito.verify(replicaManagerMock).resizeFetcherThreadPool(newSize = 2) Mockito.verify(replicaManagerMock).resizeFetcherThreadPool(newSize = 2)
props.put(KafkaConfig.NumRecoveryThreadsPerDataDirProp, "2") props.put(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, "2")
config.dynamicConfig.updateDefaultConfig(props) config.dynamicConfig.updateDefaultConfig(props)
assertEquals(2, config.numRecoveryThreadsPerDataDir) assertEquals(2, config.numRecoveryThreadsPerDataDir)
Mockito.verify(logManagerMock).resizeRecoveryThreadPool(newSize = 2) Mockito.verify(logManagerMock).resizeRecoveryThreadPool(newSize = 2)
@ -219,7 +219,7 @@ class DynamicBrokerConfigTest {
val invalidProps = Map(CleanerConfig.LOG_CLEANER_THREADS_PROP -> "invalid") val invalidProps = Map(CleanerConfig.LOG_CLEANER_THREADS_PROP -> "invalid")
verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, invalidProps) verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, invalidProps)
val excludedTopicConfig = Map(KafkaConfig.LogMessageFormatVersionProp -> "0.10.2") val excludedTopicConfig = Map(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG -> "0.10.2")
verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, excludedTopicConfig) verifyConfigUpdateWithInvalidConfig(config, origProps, validProps, excludedTopicConfig)
} }
@ -604,8 +604,8 @@ class DynamicBrokerConfigTest {
DynamicBrokerConfig.brokerConfigSynonyms("listener.name.sasl_ssl.plain.sasl.jaas.config", matchListenerOverride = true)) DynamicBrokerConfig.brokerConfigSynonyms("listener.name.sasl_ssl.plain.sasl.jaas.config", matchListenerOverride = true))
assertEquals(List("some.config"), assertEquals(List("some.config"),
DynamicBrokerConfig.brokerConfigSynonyms("some.config", matchListenerOverride = true)) DynamicBrokerConfig.brokerConfigSynonyms("some.config", matchListenerOverride = true))
assertEquals(List(KafkaConfig.LogRollTimeMillisProp, KafkaConfig.LogRollTimeHoursProp), assertEquals(List(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG),
DynamicBrokerConfig.brokerConfigSynonyms(KafkaConfig.LogRollTimeMillisProp, matchListenerOverride = true)) DynamicBrokerConfig.brokerConfigSynonyms(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, matchListenerOverride = true))
} }
@Test @Test
@ -721,7 +721,7 @@ class DynamicBrokerConfigTest {
@Test @Test
def testDynamicLogLocalRetentionMsConfig(): Unit = { def testDynamicLogLocalRetentionMsConfig(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.put(KafkaConfig.LogRetentionTimeMillisProp, "2592000000") props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "2592000000")
val config = KafkaConfig(props) val config = KafkaConfig(props)
val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer])) val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer]))
config.dynamicConfig.initialize(None, None) config.dynamicConfig.initialize(None, None)
@ -744,7 +744,7 @@ class DynamicBrokerConfigTest {
@Test @Test
def testDynamicLogLocalRetentionSizeConfig(): Unit = { def testDynamicLogLocalRetentionSizeConfig(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.put(KafkaConfig.LogRetentionBytesProp, "4294967296") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "4294967296")
val config = KafkaConfig(props) val config = KafkaConfig(props)
val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer])) val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer]))
config.dynamicConfig.initialize(None, None) config.dynamicConfig.initialize(None, None)
@ -820,8 +820,8 @@ class DynamicBrokerConfigTest {
logLocalRetentionBytes: Long, logLocalRetentionBytes: Long,
retentionBytes: Long): Unit = { retentionBytes: Long): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.put(KafkaConfig.LogRetentionTimeMillisProp, retentionMs.toString) props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, retentionMs.toString)
props.put(KafkaConfig.LogRetentionBytesProp, retentionBytes.toString) props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, retentionBytes.toString)
val config = KafkaConfig(props) val config = KafkaConfig(props)
val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer])) val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer]))
config.dynamicConfig.initialize(None, None) config.dynamicConfig.initialize(None, None)

View File

@ -44,7 +44,7 @@ import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity}
import org.apache.kafka.common.record.{CompressionType, RecordVersion} import org.apache.kafka.common.record.{CompressionType, RecordVersion}
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1 import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1
import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals} import org.apache.kafka.server.config.{ConfigType, ServerLogConfigs, ZooKeeperInternals}
import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.storage.internals.log.LogConfig
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
@ -87,7 +87,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
val op = new AlterConfigOp(new ConfigEntry(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, newVal.toString()), val op = new AlterConfigOp(new ConfigEntry(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, newVal.toString()),
SET) SET)
val resource2 = new ConfigResource(ConfigResource.Type.BROKER, "") val resource2 = new ConfigResource(ConfigResource.Type.BROKER, "")
val op2 = new AlterConfigOp(new ConfigEntry(KafkaConfig.LogFlushIntervalMsProp, newVal.toString()), val op2 = new AlterConfigOp(new ConfigEntry(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, newVal.toString()),
SET) SET)
admin.incrementalAlterConfigs(Map( admin.incrementalAlterConfigs(Map(
resource -> List(op).asJavaCollection, resource -> List(op).asJavaCollection,

View File

@ -34,6 +34,7 @@ import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.ByteUtils import org.apache.kafka.common.utils.ByteUtils
import org.apache.kafka.common.{TopicPartition, requests} import org.apache.kafka.common.{TopicPartition, requests}
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
@ -44,7 +45,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
def generateConfigs = { def generateConfigs = {
val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) val props = TestUtils.createBrokerConfig(1, zkConnectOrNull)
props.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") props.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false")
List(KafkaConfig.fromProps(props)) List(KafkaConfig.fromProps(props))
} }

View File

@ -27,6 +27,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.serialization.StringSerializer import org.apache.kafka.common.serialization.StringSerializer
import org.apache.kafka.server.config.ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -53,7 +54,7 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
override protected def brokerPropertyOverrides(properties: Properties): Unit = { override protected def brokerPropertyOverrides(properties: Properties): Unit = {
super.brokerPropertyOverrides(properties) super.brokerPropertyOverrides(properties)
properties.put(KafkaConfig.LogMessageDownConversionEnableProp, "false") properties.put(LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, "false")
} }
private def initProducer(): Unit = { private def initProducer(): Unit = {

View File

@ -75,7 +75,7 @@ import org.apache.kafka.common._
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.ClientMetricsManager
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer} import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer}
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1}
import org.apache.kafka.server.common.{Features, MetadataVersion} import org.apache.kafka.server.common.{Features, MetadataVersion}
@ -1307,7 +1307,7 @@ class KafkaApisTest extends Logging {
when(txnCoordinator.transactionTopicConfigs).thenReturn(new Properties) when(txnCoordinator.transactionTopicConfigs).thenReturn(new Properties)
true true
case _ => case _ =>
topicConfigOverride.put(KafkaConfig.NumPartitionsProp, numBrokersNeeded.toString) topicConfigOverride.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numBrokersNeeded.toString)
topicConfigOverride.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, numBrokersNeeded.toString) topicConfigOverride.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, numBrokersNeeded.toString)
false false
} }

View File

@ -28,7 +28,6 @@ import org.apache.kafka.common.record.{CompressionType, Records}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, UNKNOWN_ADDRESS_SPEC_INSTANCE} import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, UNKNOWN_ADDRESS_SPEC_INSTANCE}
import org.apache.kafka.server.config.ReplicationConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -42,7 +41,7 @@ import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, Transact
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.server.common.MetadataVersion 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.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1}
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerTopicConfigSynonyms, ZkConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.function.Executable
@ -55,7 +54,7 @@ class KafkaConfigTest {
@Test @Test
def testLogRetentionTimeHoursProvided(): Unit = { def testLogRetentionTimeHoursProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "1") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "1")
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis) assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis)
@ -64,7 +63,7 @@ class KafkaConfigTest {
@Test @Test
def testLogRetentionTimeMinutesProvided(): Unit = { def testLogRetentionTimeMinutesProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "30") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "30")
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@ -73,7 +72,7 @@ class KafkaConfigTest {
@Test @Test
def testLogRetentionTimeMsProvided(): Unit = { def testLogRetentionTimeMsProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1800000") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1800000")
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@ -90,8 +89,8 @@ class KafkaConfigTest {
@Test @Test
def testLogRetentionTimeBothMinutesAndHoursProvided(): Unit = { def testLogRetentionTimeBothMinutesAndHoursProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "30") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "30")
props.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "1") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "1")
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@ -100,8 +99,8 @@ class KafkaConfigTest {
@Test @Test
def testLogRetentionTimeBothMinutesAndMsProvided(): Unit = { def testLogRetentionTimeBothMinutesAndMsProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.LogRetentionTimeMillisProp, "1800000") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "1800000")
props.setProperty(KafkaConfig.LogRetentionTimeMinutesProp, "10") props.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "10")
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
@ -603,14 +602,14 @@ class KafkaConfigTest {
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.0") props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.0")
// We need to set the message format version to make the configuration valid. // We need to set the message format version to make the configuration valid.
props.setProperty(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.0") props.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.8.2.0")
val conf2 = KafkaConfig.fromProps(props) val conf2 = KafkaConfig.fromProps(props)
assertEquals(IBP_0_8_2, conf2.interBrokerProtocolVersion) assertEquals(IBP_0_8_2, conf2.interBrokerProtocolVersion)
// check that 0.8.2.0 is the same as 0.8.2.1 // check that 0.8.2.0 is the same as 0.8.2.1
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.1") props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.8.2.1")
// We need to set the message format version to make the configuration valid // We need to set the message format version to make the configuration valid
props.setProperty(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.1") props.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.8.2.1")
val conf3 = KafkaConfig.fromProps(props) val conf3 = KafkaConfig.fromProps(props)
assertEquals(IBP_0_8_2, conf3.interBrokerProtocolVersion) assertEquals(IBP_0_8_2, conf3.interBrokerProtocolVersion)
@ -664,7 +663,7 @@ class KafkaConfigTest {
@Test @Test
def testLogRollTimeMsProvided(): Unit = { def testLogRollTimeMsProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.LogRollTimeMillisProp, "1800000") props.setProperty(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, "1800000")
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis) assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis)
@ -673,8 +672,8 @@ class KafkaConfigTest {
@Test @Test
def testLogRollTimeBothMsAndHoursProvided(): Unit = { def testLogRollTimeBothMsAndHoursProvided(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.LogRollTimeMillisProp, "1800000") props.setProperty(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, "1800000")
props.setProperty(KafkaConfig.LogRollTimeHoursProp, "1") props.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "1")
val cfg = KafkaConfig.fromProps(props) val cfg = KafkaConfig.fromProps(props)
assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis) assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis)
@ -745,7 +744,7 @@ class KafkaConfigTest {
def buildConfig(interBrokerProtocol: MetadataVersion, messageFormat: MetadataVersion): KafkaConfig = { def buildConfig(interBrokerProtocol: MetadataVersion, messageFormat: MetadataVersion): KafkaConfig = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, interBrokerProtocol.version) props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, interBrokerProtocol.version)
props.setProperty(KafkaConfig.LogMessageFormatVersionProp, messageFormat.version) props.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, messageFormat.version)
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@ -824,7 +823,7 @@ class KafkaConfigTest {
case KafkaConfig.MetadataMaxIdleIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KafkaConfig.MetadataMaxIdleIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.AuthorizerClassNameProp => //ignore string case KafkaConfig.AuthorizerClassNameProp => //ignore string
case KafkaConfig.CreateTopicPolicyClassNameProp => //ignore string case ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG => //ignore string
case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
@ -834,21 +833,21 @@ class KafkaConfigTest {
case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.FailedAuthenticationDelayMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1") case KafkaConfig.FailedAuthenticationDelayMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1")
case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.NUM_PARTITIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogDirsProp => // ignore string case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore string
case KafkaConfig.LogDirProp => // ignore string case ServerLogConfigs.LOG_DIR_CONFIG => // ignore string
case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1) case ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1)
case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0") case ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0")
case CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") case CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "1024") case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", "1024")
case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") case CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
@ -857,17 +856,17 @@ class KafkaConfigTest {
case CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") case CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
case CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") case CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
case CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number") case CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "3") case ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "3")
case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.LogMessageTimestampDifferenceMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.LogMessageTimestampBeforeMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.LogMessageTimestampAfterMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.LogFlushStartOffsetCheckpointIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") case ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0")
case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
@ -1152,12 +1151,12 @@ class KafkaConfigTest {
defaults.setProperty(KafkaConfig.BrokerIdProp, "1") defaults.setProperty(KafkaConfig.BrokerIdProp, "1")
defaults.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:1122") defaults.setProperty(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:1122")
defaults.setProperty(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3") defaults.setProperty(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3")
defaults.setProperty(KafkaConfig.LogDirProp, "/tmp1,/tmp2") defaults.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, "/tmp1,/tmp2")
defaults.setProperty(KafkaConfig.LogRollTimeHoursProp, "12") defaults.setProperty(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG, "12")
defaults.setProperty(KafkaConfig.LogRollTimeJitterHoursProp, "11") defaults.setProperty(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG, "11")
defaults.setProperty(KafkaConfig.LogRetentionTimeHoursProp, "10") defaults.setProperty(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "10")
//For LogFlushIntervalMsProp //For LOG_FLUSH_INTERVAL_MS_CONFIG
defaults.setProperty(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") defaults.setProperty(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG, "123")
defaults.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_COMPRESSION_CODEC_CONFIG, CompressionType.SNAPPY.id.toString) defaults.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_COMPRESSION_CODEC_CONFIG, CompressionType.SNAPPY.id.toString)
// For MetricRecordingLevelProp // For MetricRecordingLevelProp
defaults.setProperty(KafkaConfig.MetricRecordingLevelProp, Sensor.RecordingLevel.DEBUG.toString) defaults.setProperty(KafkaConfig.MetricRecordingLevelProp, Sensor.RecordingLevel.DEBUG.toString)
@ -1493,7 +1492,7 @@ class KafkaConfigTest {
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir) props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir)
props.setProperty(KafkaConfig.LogDirProp, dataDir) props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, dataDir)
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
@ -1511,7 +1510,7 @@ class KafkaConfigTest {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.LogDirProp, s"$dataDir1,$dataDir2") props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, s"$dataDir1,$dataDir2")
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
@ -1861,7 +1860,7 @@ class KafkaConfigTest {
def testMultipleLogDirectoriesNotSupportedWithRemoteLogStorage(): Unit = { def testMultipleLogDirectoriesNotSupportedWithRemoteLogStorage(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true)) props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true))
props.put(KafkaConfig.LogDirsProp, "/tmp/a,/tmp/b") props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a,/tmp/b")
val caught = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) val caught = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
assertTrue(caught.getMessage.contains("Multiple log directories `/tmp/a,/tmp/b` are not supported when remote log storage is enabled")) assertTrue(caught.getMessage.contains("Multiple log directories `/tmp/a,/tmp/b` are not supported when remote log storage is enabled"))
@ -1871,7 +1870,7 @@ class KafkaConfigTest {
def testSingleLogDirectoryWithRemoteLogStorage(): Unit = { def testSingleLogDirectoryWithRemoteLogStorage(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true)) props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true))
props.put(KafkaConfig.LogDirsProp, "/tmp/a") props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a")
assertDoesNotThrow(() => KafkaConfig.fromProps(props)) assertDoesNotThrow(() => KafkaConfig.fromProps(props))
} }
} }

View File

@ -24,7 +24,7 @@ import org.apache.kafka.common.{KafkaException, Uuid}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.test.TestUtils import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -90,7 +90,7 @@ class KafkaRaftServerTest {
try { try {
writeMetaProperties(tempLogDir, metaProperties) writeMetaProperties(tempLogDir, metaProperties)
metadataVersion.foreach(mv => writeBootstrapMetadata(tempLogDir, mv)) metadataVersion.foreach(mv => writeBootstrapMetadata(tempLogDir, mv))
configProperties.put(KafkaConfig.LogDirProp, tempLogDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, tempLogDir.getAbsolutePath)
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, "") KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, "")
} finally { } finally {
@ -130,7 +130,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ProcessRolesProp, "broker") configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092") configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.LogDirProp, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(",")) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
@ -158,7 +158,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092") configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.MetadataLogDirProp, invalidDir.getAbsolutePath) configProperties.put(KafkaConfig.MetadataLogDirProp, invalidDir.getAbsolutePath)
configProperties.put(KafkaConfig.LogDirProp, validDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, validDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
@ -188,7 +188,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092") configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, validDir.getAbsolutePath) configProperties.put(KafkaConfig.MetadataLogDirProp, validDir.getAbsolutePath)
configProperties.put(KafkaConfig.LogDirProp, invalidDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, invalidDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
@ -224,7 +224,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092") configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, metadataDir.getAbsolutePath) configProperties.put(KafkaConfig.MetadataLogDirProp, metadataDir.getAbsolutePath)
configProperties.put(KafkaConfig.LogDirProp, dataDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, dataDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
@ -251,7 +251,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ProcessRolesProp, "broker") configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092") configProperties.put(KafkaConfig.QuorumVotersProp, s"${(nodeId + 1)}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.LogDirProp, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(",")) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
@ -307,7 +307,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093") configProperties.put(KafkaConfig.ListenersProp, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093")
configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
configProperties.put(KafkaConfig.LogDirProp, logDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
val (metaPropertiesEnsemble, bootstrapMetadata) = val (metaPropertiesEnsemble, bootstrapMetadata) =
invokeLoadMetaProperties(metaProperties, configProperties, None) invokeLoadMetaProperties(metaProperties, configProperties, None)

View File

@ -28,7 +28,7 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{KafkaStorageException, NotLeaderOrFollowerException} import org.apache.kafka.common.errors.{KafkaStorageException, NotLeaderOrFollowerException}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
@ -86,7 +86,7 @@ class LogDirFailureTest extends IntegrationTestHarness {
try { try {
val props = TestUtils.createBrokerConfig(brokerCount, zkConnect, logDirCount = 3) val props = TestUtils.createBrokerConfig(brokerCount, zkConnect, logDirCount = 3)
props.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.11.0") props.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.11.0")
props.put(KafkaConfig.LogMessageFormatVersionProp, "0.11.0") props.put(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
val kafkaConfig = KafkaConfig.fromProps(props) val kafkaConfig = KafkaConfig.fromProps(props)
val logDir = new File(kafkaConfig.logDirs.head) val logDir = new File(kafkaConfig.logDirs.head)
// Make log directory of the partition on the leader broker inaccessible by replacing it with a file // Make log directory of the partition on the leader broker inaccessible by replacing it with a file

View File

@ -40,7 +40,7 @@ import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.LeaderRecoveryState
import org.apache.kafka.metadata.PartitionRegistration import org.apache.kafka.metadata.PartitionRegistration
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesVersion} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesVersion}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.util.{MockTime, ShutdownableThread} import org.apache.kafka.server.util.{MockTime, ShutdownableThread}
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel}
@ -170,7 +170,7 @@ class ReplicaManagerConcurrencyTest extends Logging {
props.put(KafkaConfig.ProcessRolesProp, "broker") props.put(KafkaConfig.ProcessRolesProp, "broker")
props.put(KafkaConfig.NodeIdProp, localId.toString) props.put(KafkaConfig.NodeIdProp, localId.toString)
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") props.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.put(KafkaConfig.LogDirProp, logDir.getAbsolutePath) props.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
props.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, 5000.toString) props.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, 5000.toString)
val config = new KafkaConfig(props, doLog = false) val config = new KafkaConfig(props, doLog = false)

View File

@ -76,7 +76,7 @@ import org.apache.kafka.common.config.{AbstractConfig, TopicConfig}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.timer.MockTimer import org.apache.kafka.server.util.timer.MockTimer
import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile
import org.mockito.invocation.InvocationOnMock import org.mockito.invocation.InvocationOnMock
@ -4794,7 +4794,7 @@ class ReplicaManagerTest {
val topicPartition = new TopicPartition(topic, 0) val topicPartition = new TopicPartition(topic, 0)
val replicaManager = setupReplicaManagerWithMockedPurgatories( val replicaManager = setupReplicaManagerWithMockedPurgatories(
timer = new MockTimer(time), timer = new MockTimer(time),
propsModifier = props => props.put(KafkaConfig.LogDirsProp, dataDir.getAbsolutePath) propsModifier = props => props.put(ServerLogConfigs.LOG_DIRS_CONFIG, dataDir.getAbsolutePath)
) )
try { try {
@ -5606,7 +5606,7 @@ class ReplicaManagerTest {
val replicaManager = setupReplicaManagerWithMockedPurgatories( val replicaManager = setupReplicaManagerWithMockedPurgatories(
timer = new MockTimer(time), timer = new MockTimer(time),
brokerId = localId, brokerId = localId,
propsModifier = props => props.put(KafkaConfig.LogDirsProp, dataDir.getAbsolutePath), propsModifier = props => props.put(ServerLogConfigs.LOG_DIRS_CONFIG, dataDir.getAbsolutePath),
enableRemoteStorage = true enableRemoteStorage = true
) )

View File

@ -38,7 +38,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer} import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.BrokerState import org.apache.kafka.metadata.BrokerState
import org.apache.kafka.server.config.ZkConfigs import org.apache.kafka.server.config.{ServerLogConfigs, ZkConfigs}
import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout} import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.function.Executable
@ -64,11 +64,11 @@ class ServerShutdownTest extends KafkaServerTestHarness {
priorConfig.foreach { config => priorConfig.foreach { config =>
// keep the same log directory // keep the same log directory
val originals = config.originals val originals = config.originals
val logDirsValue = originals.get(KafkaConfig.LogDirsProp) val logDirsValue = originals.get(ServerLogConfigs.LOG_DIRS_CONFIG)
if (logDirsValue != null) { if (logDirsValue != null) {
propsToChangeUponRestart.put(KafkaConfig.LogDirsProp, logDirsValue) propsToChangeUponRestart.put(ServerLogConfigs.LOG_DIRS_CONFIG, logDirsValue)
} else { } else {
propsToChangeUponRestart.put(KafkaConfig.LogDirProp, originals.get(KafkaConfig.LogDirProp)) propsToChangeUponRestart.put(ServerLogConfigs.LOG_DIR_CONFIG, originals.get(ServerLogConfigs.LOG_DIR_CONFIG))
} }
} }
priorConfig = Some(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, zkConnectOrNull).head, propsToChangeUponRestart)) priorConfig = Some(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, zkConnectOrNull).head, propsToChangeUponRestart))

View File

@ -19,7 +19,7 @@ package kafka.server.epoch
import kafka.log.UnifiedLog import kafka.log.UnifiedLog
import kafka.server.KafkaConfig._ import kafka.server.KafkaConfig._
import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness} import kafka.server.{KafkaServer, QuorumTestHarness}
import kafka.tools.DumpLogSegments import kafka.tools.DumpLogSegments
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import kafka.utils.{CoreUtils, Logging, TestUtils} import kafka.utils.{CoreUtils, Logging, TestUtils}
@ -28,7 +28,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.serialization.ByteArrayDeserializer
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
import org.apache.kafka.storage.internals.log.EpochEntry import org.apache.kafka.storage.internals.log.EpochEntry
@ -178,7 +178,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
assertEquals(getLogFile(brokers(0), 0).length, getLogFile(brokers(1), 0).length, "Log files should match Broker0 vs Broker 1") assertEquals(getLogFile(brokers(0), 0).length, getLogFile(brokers(1), 0).length, "Log files should match Broker0 vs Broker 1")
} }
//We can reproduce the pre-KIP-101 failure of this test by setting KafkaConfig.INTER_BROKER_PROTOCOL_VERSION_PROP = IBP_0_11_0_IV1 //We can reproduce the pre-KIP-101 failure of this test by setting ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG = IBP_0_11_0_IV1
@Test @Test
def offsetsShouldNotGoBackwards(): Unit = { def offsetsShouldNotGoBackwards(): Unit = {
@ -303,7 +303,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
// A single partition topic with 2 replicas, min.isr = 1 // A single partition topic with 2 replicas, min.isr = 1
TestUtils.createTopic(zkClient, topic, Map(0 -> Seq(100, 101)), brokers, TestUtils.createTopic(zkClient, topic, Map(0 -> Seq(100, 101)), brokers,
CoreUtils.propsWith((KafkaConfig.MinInSyncReplicasProp, "1"))) CoreUtils.propsWith((ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "1")))
producer = TestUtils.createProducer(plaintextBootstrapServers(brokers), acks = 1) producer = TestUtils.createProducer(plaintextBootstrapServers(brokers), acks = 1)

View File

@ -37,6 +37,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.metadata.MetadataRecordSerde
import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch} import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch}
import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
import org.apache.kafka.snapshot.RecordsSnapshotWriter import org.apache.kafka.snapshot.RecordsSnapshotWriter
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
@ -315,7 +316,7 @@ class DumpLogSegmentsTest {
retentionMillis = 60 * 1000, retentionMillis = 60 * 1000,
maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES, maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES,
maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES, maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES,
fileDeleteDelayMs = LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
nodeId = 1 nodeId = 1
) )
) )

View File

@ -31,6 +31,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.common.metadata.UserScramCredentialRecord import org.apache.kafka.common.metadata.UserScramCredentialRecord
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue}
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -44,7 +45,7 @@ class StorageToolTest {
private def newSelfManagedProperties() = { private def newSelfManagedProperties() = {
val properties = new Properties() val properties = new Properties()
properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo,/tmp/bar") properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo,/tmp/bar")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller") properties.setProperty(KafkaConfig.ProcessRolesProp, "controller")
properties.setProperty(KafkaConfig.NodeIdProp, "2") properties.setProperty(KafkaConfig.NodeIdProp, "2")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9092") properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9092")
@ -407,7 +408,7 @@ Found problem:
val propsFile = TestUtils.tempFile() val propsFile = TestUtils.tempFile()
val propsStream = Files.newOutputStream(propsFile.toPath) val propsStream = Files.newOutputStream(propsFile.toPath)
// This test does format the directory specified so use a tempdir // This test does format the directory specified so use a tempdir
properties.setProperty(KafkaConfig.LogDirsProp, TestUtils.tempDir().toString) properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString)
properties.store(propsStream, "config.props") properties.store(propsStream, "config.props")
propsStream.close() propsStream.close()
@ -461,7 +462,7 @@ Found problem:
val propsFile = TestUtils.tempFile() val propsFile = TestUtils.tempFile()
val propsStream = Files.newOutputStream(propsFile.toPath) val propsStream = Files.newOutputStream(propsFile.toPath)
try { try {
properties.setProperty(KafkaConfig.LogDirsProp, TestUtils.tempDir().toString) properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString)
properties.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, enableUnstable.toString) properties.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, enableUnstable.toString)
properties.store(propsStream, "config.props") properties.store(propsStream, "config.props")
} finally { } finally {

View File

@ -75,7 +75,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.metadata.properties.MetaProperties import org.apache.kafka.metadata.properties.MetaProperties
import org.apache.kafka.server.{ClientMetricsManager, ControllerRequestCompletionHandler} import org.apache.kafka.server.{ClientMetricsManager, ControllerRequestCompletionHandler}
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
import org.apache.kafka.server.config.ZkConfigs import org.apache.kafka.server.config.ZkConfigs
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
@ -351,9 +351,9 @@ object TestUtils extends Logging {
// We can verify this by using a mixture of relative path and absolute path as log directories in the test // We can verify this by using a mixture of relative path and absolute path as log directories in the test
if (i % 2 == 0) tempDir().getAbsolutePath else tempRelativeDir("data") if (i % 2 == 0) tempDir().getAbsolutePath else tempRelativeDir("data")
).mkString(",") ).mkString(",")
props.put(KafkaConfig.LogDirsProp, logDirs) props.put(ServerLogConfigs.LOG_DIRS_CONFIG, logDirs)
} else { } else {
props.put(KafkaConfig.LogDirProp, tempDir().getAbsolutePath) props.put(ServerLogConfigs.LOG_DIR_CONFIG, tempDir().getAbsolutePath)
} }
if (zkConnect == null) { if (zkConnect == null) {
props.put(KafkaConfig.ProcessRolesProp, "broker") props.put(KafkaConfig.ProcessRolesProp, "broker")
@ -370,7 +370,7 @@ object TestUtils extends Logging {
props.put(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, "1500") props.put(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, "1500")
props.put(KafkaConfig.ControlledShutdownEnableProp, enableControlledShutdown.toString) props.put(KafkaConfig.ControlledShutdownEnableProp, enableControlledShutdown.toString)
props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString) props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString)
props.put(KafkaConfig.LogDeleteDelayMsProp, "1000") props.put(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000")
props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100") props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100")
props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152") props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152")
props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1")
@ -396,7 +396,7 @@ object TestUtils extends Logging {
if (enableToken) if (enableToken)
props.put(KafkaConfig.DelegationTokenSecretKeyProp, "secretkey") props.put(KafkaConfig.DelegationTokenSecretKeyProp, "secretkey")
props.put(KafkaConfig.NumPartitionsProp, numPartitions.toString) props.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numPartitions.toString)
props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, defaultReplicationFactor.toString) props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, defaultReplicationFactor.toString)
if (enableFetchFromFollower) { if (enableFetchFromFollower) {
@ -411,7 +411,7 @@ object TestUtils extends Logging {
config.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, version.version) config.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, version.version)
// for clarity, only set the log message format version if it's not ignored // for clarity, only set the log message format version if it's not ignored
if (!LogConfig.shouldIgnoreMessageFormatVersion(version)) if (!LogConfig.shouldIgnoreMessageFormatVersion(version))
config.setProperty(KafkaConfig.LogMessageFormatVersionProp, version.version) config.setProperty(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, version.version)
} }
def createAdminClient[B <: KafkaBroker]( def createAdminClient[B <: KafkaBroker](

View File

@ -0,0 +1,188 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.server.config;
import org.apache.kafka.common.config.TopicConfig;
import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1;
import static org.apache.kafka.server.config.ServerTopicConfigSynonyms.LOG_PREFIX;
/**
* Common home for broker-side log configs which need to be accessible from the libraries shared
* between the broker and the multiple modules in Kafka.
*
* Note this is an internal API and subject to change without notice.
*/
public class ServerLogConfigs {
public static final String NUM_PARTITIONS_CONFIG = "num.partitions";
public static final int NUM_PARTITIONS_DEFAULT = 1;
public static final String NUM_PARTITIONS_DOC = "The default number of log partitions per topic";
public static final String LOG_DIRS_CONFIG = LOG_PREFIX + "dirs";
public static final String LOG_DIR_CONFIG = LOG_PREFIX + "dir";
public static final String LOG_DIR_DEFAULT = "/tmp/kafka-logs";
public static final String LOG_DIR_DOC = "The directory in which the log data is kept (supplemental for " + LOG_DIRS_CONFIG + " property)";
public static final String LOG_DIRS_DOC = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LOG_DIR_CONFIG + " is used.";
public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG);
public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file";
public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG);
public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours";
public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used";
public static final String LOG_ROLL_TIME_HOURS_DOC = "The maximum time before a new log segment is rolled out (in hours), secondary to " + LOG_ROLL_TIME_MILLIS_CONFIG + " property";
public static final String LOG_ROLL_TIME_JITTER_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_JITTER_MS_CONFIG);
public static final String LOG_ROLL_TIME_JITTER_HOURS_CONFIG = LOG_PREFIX + "roll.jitter.hours";
public static final String LOG_ROLL_TIME_JITTER_MILLIS_DOC = "The maximum jitter to subtract from logRollTimeMillis (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_JITTER_HOURS_CONFIG + " is used";
public static final String LOG_ROLL_TIME_JITTER_HOURS_DOC = "The maximum jitter to subtract from logRollTimeMillis (in hours), secondary to " + LOG_ROLL_TIME_JITTER_MILLIS_CONFIG + " property";
public static final String LOG_RETENTION_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_MS_CONFIG);
public static final String LOG_RETENTION_TIME_MINUTES_CONFIG = LOG_PREFIX + "retention.minutes";
public static final String LOG_RETENTION_TIME_HOURS_CONFIG = LOG_PREFIX + "retention.hours";
public static final String LOG_RETENTION_TIME_MILLIS_DOC = "The number of milliseconds to keep a log file before deleting it (in milliseconds), If not set, the value in " + LOG_RETENTION_TIME_MINUTES_CONFIG + " is used. If set to -1, no time limit is applied.";
public static final String LOG_RETENTION_TIME_MINUTES_DOC = "The number of minutes to keep a log file before deleting it (in minutes), secondary to " + LOG_RETENTION_TIME_MILLIS_CONFIG + " property. If not set, the value in " + LOG_RETENTION_TIME_HOURS_CONFIG + " is used";
public static final String LOG_RETENTION_TIME_HOURS_DOC = "The number of hours to keep a log file before deleting it (in hours), tertiary to " + LOG_RETENTION_TIME_MILLIS_CONFIG + " property";
public static final String LOG_RETENTION_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.RETENTION_BYTES_CONFIG);
public static final long LOG_RETENTION_BYTES_DEFAULT = -1L;
public static final String LOG_RETENTION_BYTES_DOC = "The maximum size of the log before deleting it";
public static final String LOG_CLEANUP_INTERVAL_MS_CONFIG = LOG_PREFIX + "retention.check.interval.ms";
public static final long LOG_CLEANUP_INTERVAL_MS_DEFAULT = 5 * 60 * 1000L;
public static final String LOG_CLEANUP_INTERVAL_MS_DOC = "The frequency in milliseconds that the log cleaner checks whether any log is eligible for deletion";
public static final String LOG_CLEANUP_POLICY_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG);
public static final String LOG_CLEANUP_POLICY_DEFAULT = TopicConfig.CLEANUP_POLICY_DELETE;
public static final String LOG_CLEANUP_POLICY_DOC = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies. Valid policies are: \"delete\" and \"compact\"";
public static final String LOG_INDEX_SIZE_MAX_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG);
public static final int LOG_INDEX_SIZE_MAX_BYTES_DEFAULT = 10 * 1024 * 1024;
public static final String LOG_INDEX_SIZE_MAX_BYTES_DOC = "The maximum size in bytes of the offset index";
public static final String LOG_INDEX_INTERVAL_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG);
public static final int LOG_INDEX_INTERVAL_BYTES_DEFAULT = 4096;
public static final String LOG_INDEX_INTERVAL_BYTES_DOC = "The interval with which we add an entry to the offset index.";
public static final String LOG_FLUSH_INTERVAL_MESSAGES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG);
public static final long LOG_FLUSH_INTERVAL_MESSAGES_DEFAULT = Long.MAX_VALUE;
public static final String LOG_FLUSH_INTERVAL_MESSAGES_DOC = "The number of messages accumulated on a log partition before messages are flushed to disk.";
public static final String LOG_DELETE_DELAY_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG);
public static final long LOG_DELETE_DELAY_MS_DEFAULT = 60000L;
public static final String LOG_DELETE_DELAY_MS_DOC = "The amount of time to wait before deleting a file from the filesystem";
public static final String LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG = LOG_PREFIX + "flush.scheduler.interval.ms";
public static final long LOG_FLUSH_SCHEDULER_INTERVAL_MS_DEFAULT = Long.MAX_VALUE;
public static final String LOG_FLUSH_SCHEDULER_INTERVAL_MS_DOC = "The frequency in ms that the log flusher checks whether any log needs to be flushed to disk";
public static final String LOG_FLUSH_INTERVAL_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.FLUSH_MS_CONFIG);
public static final String LOG_FLUSH_INTERVAL_MS_DOC = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk. If not set, the value in " + LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG + " is used";
public static final String LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG = LOG_PREFIX + "flush.offset.checkpoint.interval.ms";
public static final int LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT = 60000;
public static final String LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_DOC = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point.";
public static final String LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG = LOG_PREFIX + "flush.start.offset.checkpoint.interval.ms";
public static final int LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DEFAULT = 60000;
public static final String LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_DOC = "The frequency with which we update the persistent record of log start offset";
public static final String LOG_PRE_ALLOCATE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.PREALLOCATE_CONFIG);
public static final String LOG_PRE_ALLOCATE_ENABLE_DOC = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true.";
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
/**
* @deprecated since "3.0"
*/
@Deprecated
public static final String LOG_MESSAGE_FORMAT_VERSION_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG);
@Deprecated
public static final String LOG_MESSAGE_FORMAT_VERSION_DEFAULT = IBP_3_0_IV1.version();
public static final String LOG_MESSAGE_FORMAT_VERSION_DOC = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid MetadataVersion. " +
"Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check MetadataVersion for more details. By setting a particular message format version, the " +
"user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " +
"will cause consumers with older versions to break as they will receive messages with a format that they don't understand.";
public static final String LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG);
public static final String LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT = "CreateTime";
public static final String LOG_MESSAGE_TIMESTAMP_TYPE_DOC = "Define whether the timestamp in the message is message create time or log append time. The value should be either " +
"<code>CreateTime</code> or <code>LogAppendTime</code>.";
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
/**
* @deprecated since "3.6"
*/
@Deprecated
public static final String LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG);
@Deprecated
public static final long LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT = Long.MAX_VALUE;
public static final String LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC = "[DEPRECATED] The maximum difference allowed between the timestamp when a broker receives " +
"a message and the timestamp specified in the message. If log.message.timestamp.type=CreateTime, a message will be rejected " +
"if the difference in timestamp exceeds this threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." +
"The maximum timestamp difference allowed should be no greater than log.retention.ms to avoid unnecessarily frequent log rolling.";
public static final String LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG);
public static final long LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT = Long.MAX_VALUE;
public static final String LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC = "This configuration sets the allowable timestamp difference between the " +
"broker's timestamp and the message timestamp. The message timestamp can be earlier than or equal to the broker's " +
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime.";
public static final String LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG);
public static final long LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT = Long.MAX_VALUE;
public static final String LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC = "This configuration sets the allowable timestamp difference between the " +
"message timestamp and the broker's timestamp. The message timestamp can be later than or equal to the broker's " +
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime.";
public static final String NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG = "num.recovery.threads.per.data.dir";
public static final int NUM_RECOVERY_THREADS_PER_DATA_DIR_DEFAULT = 1;
public static final String NUM_RECOVERY_THREADS_PER_DATA_DIR_DOC = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown";
public static final String AUTO_CREATE_TOPICS_ENABLE_CONFIG = "auto.create.topics.enable";
public static final boolean AUTO_CREATE_TOPICS_ENABLE_DEFAULT = true;
public static final String AUTO_CREATE_TOPICS_ENABLE_DOC = "Enable auto creation of topic on the server.";
public static final String MIN_IN_SYNC_REPLICAS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG);
public static final int MIN_IN_SYNC_REPLICAS_DEFAULT = 1;
public static final String MIN_IN_SYNC_REPLICAS_DOC = "When a producer sets acks to \"all\" (or \"-1\"), " +
"<code>min.insync.replicas</code> specifies the minimum number of replicas that must acknowledge " +
"a write for the write to be considered successful. If this minimum cannot be met, " +
"then the producer will raise an exception (either <code>NotEnoughReplicas</code> or " +
"<code>NotEnoughReplicasAfterAppend</code>).<br>When used together, <code>min.insync.replicas</code> and acks " +
"allow you to enforce greater durability guarantees. A typical scenario would be to " +
"create a topic with a replication factor of 3, set <code>min.insync.replicas</code> to 2, and " +
"produce with acks of \"all\". This will ensure that the producer raises an exception " +
"if a majority of replicas do not receive a write.";
public static final String CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG = "create.topic.policy.class.name";
public static final String CREATE_TOPIC_POLICY_CLASS_NAME_DOC = "The create topic policy class that should be used for validation. The class should " +
"implement the <code>org.apache.kafka.server.policy.CreateTopicPolicy</code> interface.";
public static final String ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG = "alter.config.policy.class.name";
public static final String ALTER_CONFIG_POLICY_CLASS_NAME_DOC = "The alter configs policy class that should be used for validation. The class should " +
"implement the <code>org.apache.kafka.server.policy.AlterConfigPolicy</code> interface.";
public static final String LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG);
public static final boolean LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT = true;
public static final String LOG_MESSAGE_DOWNCONVERSION_ENABLE_DOC = "This configuration controls whether " +
"down-conversion of message formats is enabled to satisfy consume requests. When set to <code>false</code>, " +
"broker will not perform down-conversion for consumers expecting an older message format. The broker responds " +
"with <code>UNSUPPORTED_VERSION</code> error for consume requests from such older clients. This configuration" +
"does not apply to any message format conversion that might be required for replication to followers.";
}

View File

@ -29,7 +29,7 @@ import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
public final class ServerTopicConfigSynonyms { public final class ServerTopicConfigSynonyms {
private static final String LOG_PREFIX = "log."; public static final String LOG_PREFIX = "log.";
public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner.";
/** /**

View File

@ -77,15 +77,6 @@ public class Defaults {
public static final long CONNECTION_SETUP_TIMEOUT_MAX_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS; public static final long CONNECTION_SETUP_TIMEOUT_MAX_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS;
public static final int FAILED_AUTHENTICATION_DELAY_MS = 100; public static final int FAILED_AUTHENTICATION_DELAY_MS = 100;
/** ********* Log Configuration *********/
public static final int NUM_PARTITIONS = 1;
public static final String LOG_DIR = "/tmp/kafka-logs";
public static final long LOG_CLEANUP_INTERVAL_MS = 5 * 60 * 1000L;
public static final int LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS = 60000;
public static final int LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS = 60000;
public static final int NUM_RECOVERY_THREADS_PER_DATA_DIR = 1;
public static final boolean AUTO_CREATE_TOPICS_ENABLE = true;
/** ********* Controlled shutdown configuration *********/ /** ********* Controlled shutdown configuration *********/
public static final int CONTROLLED_SHUTDOWN_MAX_RETRIES = 3; public static final int CONTROLLED_SHUTDOWN_MAX_RETRIES = 3;
public static final int CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS = 5000; public static final int CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS = 5000;

View File

@ -56,6 +56,7 @@ import org.apache.kafka.common.utils.ConfigUtils;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.common.MetadataVersionValidator; import org.apache.kafka.server.common.MetadataVersionValidator;
import org.apache.kafka.server.config.ServerLogConfigs;
import org.apache.kafka.server.config.ServerTopicConfigSynonyms; import org.apache.kafka.server.config.ServerTopicConfigSynonyms;
import org.apache.kafka.server.record.BrokerCompressionType; import org.apache.kafka.server.record.BrokerCompressionType;
@ -159,29 +160,15 @@ public class LogConfig extends AbstractConfig {
public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L; 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_SEGMENT_JITTER_MS = 0;
public static final long DEFAULT_RETENTION_MS = 24 * 7 * 60 * 60 * 1000L; 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_DELETE_RETENTION_MS = 24 * 60 * 60 * 1000L;
public static final long DEFAULT_MIN_COMPACTION_LAG_MS = 0; 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 long DEFAULT_MAX_COMPACTION_LAG_MS = Long.MAX_VALUE;
public static final double DEFAULT_MIN_CLEANABLE_DIRTY_RATIO = 0.5; public static final double DEFAULT_MIN_CLEANABLE_DIRTY_RATIO = 0.5;
public static final boolean DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE = false; 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 String DEFAULT_COMPRESSION_TYPE = BrokerCompressionType.PRODUCER.name;
public static final boolean DEFAULT_PREALLOCATE = false; public static final boolean DEFAULT_PREALLOCATE = false;
public static final String DEFAULT_MESSAGE_TIMESTAMP_TYPE = "CreateTime";
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
@Deprecated @Deprecated
public static final long DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS = Long.MAX_VALUE; public static final long DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS = ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT;
public static final long DEFAULT_MESSAGE_TIMESTAMP_BEFORE_MAX_MS = Long.MAX_VALUE;
public static final long DEFAULT_MESSAGE_TIMESTAMP_AFTER_MAX_MS = Long.MAX_VALUE;
public static final boolean DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE = true;
public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false; public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false;
public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes
@ -189,9 +176,10 @@ public class LogConfig extends AbstractConfig {
public static final List<String> DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList(); public static final List<String> DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList();
public static final List<String> DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList(); public static final List<String> DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList();
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */ /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details
* Keep DEFAULT_MESSAGE_FORMAT_VERSION as a way to handlee the deprecated value */
@Deprecated @Deprecated
public static final String DEFAULT_MESSAGE_FORMAT_VERSION = IBP_3_0_IV1.version(); public static final String DEFAULT_MESSAGE_FORMAT_VERSION = ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT;
// Leave these out of TopicConfig for now as they are replication quota configs // 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 LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG = "leader.replication.throttled.replicas";
@ -234,20 +222,20 @@ public class LogConfig extends AbstractConfig {
.define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_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, .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM,
TopicConfig.SEGMENT_JITTER_MS_DOC) TopicConfig.SEGMENT_JITTER_MS_DOC)
.define(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, INT, DEFAULT_SEGMENT_INDEX_BYTES, atLeast(4), MEDIUM, .define(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT, atLeast(4), MEDIUM,
TopicConfig.SEGMENT_INDEX_BYTES_DOC) TopicConfig.SEGMENT_INDEX_BYTES_DOC)
.define(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, LONG, DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), MEDIUM, .define(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_DEFAULT, atLeast(1), MEDIUM,
TopicConfig.FLUSH_MESSAGES_INTERVAL_DOC) TopicConfig.FLUSH_MESSAGES_INTERVAL_DOC)
.define(TopicConfig.FLUSH_MS_CONFIG, LONG, DEFAULT_FLUSH_MS, atLeast(0), MEDIUM, .define(TopicConfig.FLUSH_MS_CONFIG, LONG, ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.FLUSH_MS_DOC) TopicConfig.FLUSH_MS_DOC)
// can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize // can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize
.define(TopicConfig.RETENTION_BYTES_CONFIG, LONG, DEFAULT_RETENTION_BYTES, MEDIUM, TopicConfig.RETENTION_BYTES_DOC) .define(TopicConfig.RETENTION_BYTES_CONFIG, LONG, ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, MEDIUM, TopicConfig.RETENTION_BYTES_DOC)
// can be negative. See kafka.log.LogManager.cleanupExpiredSegments // can be negative. See kafka.log.LogManager.cleanupExpiredSegments
.define(TopicConfig.RETENTION_MS_CONFIG, LONG, DEFAULT_RETENTION_MS, atLeast(-1), MEDIUM, .define(TopicConfig.RETENTION_MS_CONFIG, LONG, DEFAULT_RETENTION_MS, atLeast(-1), MEDIUM,
TopicConfig.RETENTION_MS_DOC) TopicConfig.RETENTION_MS_DOC)
.define(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, INT, DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), MEDIUM, .define(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, INT, DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), MEDIUM,
TopicConfig.MAX_MESSAGE_BYTES_DOC) TopicConfig.MAX_MESSAGE_BYTES_DOC)
.define(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, INT, DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM, .define(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.INDEX_INTERVAL_BYTES_DOC) TopicConfig.INDEX_INTERVAL_BYTES_DOC)
.define(TopicConfig.DELETE_RETENTION_MS_CONFIG, LONG, DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, .define(TopicConfig.DELETE_RETENTION_MS_CONFIG, LONG, DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM,
TopicConfig.DELETE_RETENTION_MS_DOC) TopicConfig.DELETE_RETENTION_MS_DOC)
@ -255,34 +243,34 @@ public class LogConfig extends AbstractConfig {
TopicConfig.MIN_COMPACTION_LAG_MS_DOC) TopicConfig.MIN_COMPACTION_LAG_MS_DOC)
.define(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, LONG, DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, .define(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, LONG, DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM,
TopicConfig.MAX_COMPACTION_LAG_MS_DOC) TopicConfig.MAX_COMPACTION_LAG_MS_DOC)
.define(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, LONG, DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), MEDIUM, .define(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.FILE_DELETE_DELAY_MS_DOC) TopicConfig.FILE_DELETE_DELAY_MS_DOC)
.define(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, DOUBLE, DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM, .define(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, DOUBLE, DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM,
TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_DOC) TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_DOC)
.define(TopicConfig.CLEANUP_POLICY_CONFIG, LIST, DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, .define(TopicConfig.CLEANUP_POLICY_CONFIG, LIST, ServerLogConfigs.LOG_CLEANUP_POLICY_DEFAULT, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT,
TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, TopicConfig.CLEANUP_POLICY_DOC) TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, TopicConfig.CLEANUP_POLICY_DOC)
.define(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, BOOLEAN, DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE, .define(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, BOOLEAN, DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE,
MEDIUM, TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_DOC) MEDIUM, TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_DOC)
.define(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, INT, DEFAULT_MIN_IN_SYNC_REPLICAS, atLeast(1), MEDIUM, .define(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, INT, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT, atLeast(1), MEDIUM,
TopicConfig.MIN_IN_SYNC_REPLICAS_DOC) TopicConfig.MIN_IN_SYNC_REPLICAS_DOC)
.define(TopicConfig.COMPRESSION_TYPE_CONFIG, STRING, DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names().toArray(new String[0])), .define(TopicConfig.COMPRESSION_TYPE_CONFIG, STRING, DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names().toArray(new String[0])),
MEDIUM, TopicConfig.COMPRESSION_TYPE_DOC) MEDIUM, TopicConfig.COMPRESSION_TYPE_DOC)
.define(TopicConfig.PREALLOCATE_CONFIG, BOOLEAN, DEFAULT_PREALLOCATE, MEDIUM, TopicConfig.PREALLOCATE_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, .define(MESSAGE_FORMAT_VERSION_CONFIG, STRING, DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM,
MESSAGE_FORMAT_VERSION_DOC) MESSAGE_FORMAT_VERSION_DOC)
.define(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, DEFAULT_MESSAGE_TIMESTAMP_TYPE, .define(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_DEFAULT,
in("CreateTime", "LogAppendTime"), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC) in("CreateTime", "LogAppendTime"), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC)
.define(MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS, .define(MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS,
atLeast(0), MEDIUM, MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC) atLeast(0), MEDIUM, MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
.define(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_BEFORE_MAX_MS, .define(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DEFAULT,
atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC) atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_DOC)
.define(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, DEFAULT_MESSAGE_TIMESTAMP_AFTER_MAX_MS, .define(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, LONG, ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT,
atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC) atLeast(0), MEDIUM, TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_DOC)
.define(LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS, .define(LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS,
ThrottledReplicaListValidator.INSTANCE, MEDIUM, LEADER_REPLICATION_THROTTLED_REPLICAS_DOC) ThrottledReplicaListValidator.INSTANCE, MEDIUM, LEADER_REPLICATION_THROTTLED_REPLICAS_DOC)
.define(FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS, .define(FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS,
ThrottledReplicaListValidator.INSTANCE, MEDIUM, FOLLOWER_REPLICATION_THROTTLED_REPLICAS_DOC) ThrottledReplicaListValidator.INSTANCE, MEDIUM, FOLLOWER_REPLICATION_THROTTLED_REPLICAS_DOC)
.define(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, .define(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT, LOW,
TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC) TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC)
.define(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_STORAGE_ENABLE, null, .define(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, BOOLEAN, DEFAULT_REMOTE_STORAGE_ENABLE, null,
MEDIUM, TopicConfig.REMOTE_LOG_STORAGE_ENABLE_DOC) MEDIUM, TopicConfig.REMOTE_LOG_STORAGE_ENABLE_DOC)

View File

@ -16,7 +16,6 @@
*/ */
package org.apache.kafka.tiered.storage.utils; package org.apache.kafka.tiered.storage.utils;
import kafka.server.KafkaConfig;
import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.config.TopicConfig;
@ -49,6 +48,7 @@ import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CONFIG_PREFIX_PROP; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CONFIG_PREFIX_PROP;
import static org.apache.kafka.server.config.ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG;
public class TieredStorageTestUtils { public class TieredStorageTestUtils {
@ -137,7 +137,7 @@ public class TieredStorageTestUtils {
// the integration tests can confirm a given log segment is present only in the second-tier storage. // the integration tests can confirm a given log segment is present only in the second-tier storage.
// Note that this does not impact the eligibility of a log segment to be offloaded to the // Note that this does not impact the eligibility of a log segment to be offloaded to the
// second-tier storage. // second-tier storage.
overridingProps.setProperty(KafkaConfig.LogCleanupIntervalMsProp(), LOG_CLEANUP_INTERVAL_MS.toString()); overridingProps.setProperty(LOG_CLEANUP_INTERVAL_MS_CONFIG, LOG_CLEANUP_INTERVAL_MS.toString());
// The directory of the second-tier storage needs to be constant across all instances of storage managers // The directory of the second-tier storage needs to be constant across all instances of storage managers
// in every broker and throughout the test. Indeed, as brokers are restarted during the test. // in every broker and throughout the test. Indeed, as brokers are restarted during the test.
// You can override this property with a fixed path of your choice if you wish to use a non-temporary // You can override this property with a fixed path of your choice if you wish to use a non-temporary

View File

@ -25,6 +25,7 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs; import org.apache.kafka.coordinator.transaction.TransactionLogConfigs;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
import org.apache.kafka.server.config.ConfigType; import org.apache.kafka.server.config.ConfigType;
import org.apache.kafka.server.config.ServerLogConfigs;
import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.server.config.ZkConfigs;
import org.apache.kafka.server.util.MockTime; import org.apache.kafka.server.util.MockTime;
import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.CleanerConfig;
@ -121,7 +122,7 @@ public class EmbeddedKafkaCluster {
putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) 1); putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) 1);
putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 5); putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 5);
putIfAbsent(brokerConfig, TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 5); putIfAbsent(brokerConfig, TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 5);
putIfAbsent(brokerConfig, KafkaConfig.AutoCreateTopicsEnableProp(), true); putIfAbsent(brokerConfig, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, true);
for (int i = 0; i < brokers.length; i++) { for (int i = 0; i < brokers.length; i++) {
brokerConfig.put(KafkaConfig.BrokerIdProp(), i); brokerConfig.put(KafkaConfig.BrokerIdProp(), i);

View File

@ -40,6 +40,11 @@ import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import static org.apache.kafka.server.config.ServerLogConfigs.NUM_PARTITIONS_CONFIG;
import static org.apache.kafka.server.config.ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG;
import static org.apache.kafka.server.config.ServerLogConfigs.LOG_DIR_CONFIG;
/** /**
* Runs an in-memory, "embedded" instance of a Kafka broker, which listens at `127.0.0.1:9092` by * Runs an in-memory, "embedded" instance of a Kafka broker, which listens at `127.0.0.1:9092` by
* default. * default.
@ -87,14 +92,14 @@ public class KafkaEmbedded {
private Properties effectiveConfigFrom(final Properties initialConfig) { private Properties effectiveConfigFrom(final Properties initialConfig) {
final Properties effectiveConfig = new Properties(); final Properties effectiveConfig = new Properties();
effectiveConfig.put(KafkaConfig.BrokerIdProp(), 0); effectiveConfig.put(KafkaConfig.BrokerIdProp(), 0);
effectiveConfig.put(KafkaConfig.NumPartitionsProp(), 1); effectiveConfig.put(NUM_PARTITIONS_CONFIG, 1);
effectiveConfig.put(KafkaConfig.AutoCreateTopicsEnableProp(), true); effectiveConfig.put(AUTO_CREATE_TOPICS_ENABLE_CONFIG, true);
effectiveConfig.put(KafkaConfig.MessageMaxBytesProp(), 1000000); effectiveConfig.put(KafkaConfig.MessageMaxBytesProp(), 1000000);
effectiveConfig.put(KafkaConfig.ControlledShutdownEnableProp(), true); effectiveConfig.put(KafkaConfig.ControlledShutdownEnableProp(), true);
effectiveConfig.put(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, 10000); effectiveConfig.put(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, 10000);
effectiveConfig.putAll(initialConfig); effectiveConfig.putAll(initialConfig);
effectiveConfig.setProperty(KafkaConfig.LogDirProp(), logDir.getAbsolutePath()); effectiveConfig.setProperty(LOG_DIR_CONFIG, logDir.getAbsolutePath());
return effectiveConfig; return effectiveConfig;
} }