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