From 8f0003f9b694b4da5fbd2f86db872d77a43eb63f Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Thu, 5 Mar 2015 09:53:27 -0500 Subject: [PATCH] KAFKA-1845 KafkaConfig should use ConfigDef patch by Andrii Biletskyi reviewed by Gwen Shapira --- .gitignore | 1 + .../apache/kafka/common/config/ConfigDef.java | 160 ++- core/src/main/scala/kafka/Kafka.scala | 6 +- .../kafka/controller/KafkaController.scala | 2 +- .../controller/PartitionLeaderSelector.scala | 2 +- .../main/scala/kafka/server/KafkaConfig.scala | 1183 ++++++++++++----- .../main/scala/kafka/server/KafkaServer.scala | 2 +- .../kafka/server/ReplicaFetcherThread.scala | 2 +- .../kafka/api/IntegrationTestHarness.scala | 2 +- .../kafka/api/ProducerCompressionTest.scala | 2 +- .../api/ProducerFailureHandlingTest.scala | 21 +- .../kafka/api/ProducerSendTest.scala | 12 +- .../unit/kafka/admin/AddPartitionsTest.scala | 8 +- .../scala/unit/kafka/admin/AdminTest.scala | 16 +- .../kafka/admin/DeleteConsumerGroupTest.scala | 2 +- .../unit/kafka/admin/DeleteTopicTest.scala | 4 +- .../kafka/consumer/ConsumerIteratorTest.scala | 10 +- .../ZookeeperConsumerConnectorTest.scala | 12 +- .../integration/AutoOffsetResetTest.scala | 2 +- .../unit/kafka/integration/FetcherTest.scala | 2 +- .../kafka/integration/PrimitiveApiTest.scala | 2 +- .../kafka/integration/RollingBounceTest.scala | 8 +- .../kafka/integration/TopicMetadataTest.scala | 2 +- .../UncleanLeaderElectionTest.scala | 2 +- .../ZookeeperConsumerConnectorTest.scala | 15 +- .../test/scala/unit/kafka/log/LogTest.scala | 2 +- .../kafka/log4j/KafkaLog4jAppenderTest.scala | 2 +- .../unit/kafka/metrics/MetricsTest.scala | 15 +- .../kafka/producer/AsyncProducerTest.scala | 2 +- .../unit/kafka/producer/ProducerTest.scala | 4 +- .../kafka/producer/SyncProducerTest.scala | 2 +- .../kafka/server/AdvertiseBrokerTest.scala | 2 +- .../server/DynamicConfigChangeTest.scala | 2 +- .../server/HighwatermarkPersistenceTest.scala | 2 +- .../unit/kafka/server/ISRExpirationTest.scala | 16 +- .../server/KafkaConfigConfigDefTest.scala | 403 ++++++ .../unit/kafka/server/KafkaConfigTest.scala | 39 +- .../kafka/server/LeaderElectionTest.scala | 6 +- .../unit/kafka/server/LogOffsetTest.scala | 2 +- .../unit/kafka/server/LogRecoveryTest.scala | 20 +- .../unit/kafka/server/OffsetCommitTest.scala | 2 +- .../unit/kafka/server/ReplicaFetchTest.scala | 2 +- .../kafka/server/ReplicaManagerTest.scala | 6 +- .../server/ServerGenerateBrokerIdTest.scala | 10 +- .../kafka/server/ServerShutdownTest.scala | 6 +- .../unit/kafka/server/ServerStartupTest.scala | 6 +- .../unit/kafka/server/SimpleFetchTest.scala | 17 +- .../kafka/utils/ReplicationUtilsTest.scala | 2 +- 48 files changed, 1528 insertions(+), 522 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala diff --git a/.gitignore b/.gitignore index 4c6c29e2575..1f3ba7dbbf5 100644 --- a/.gitignore +++ b/.gitignore @@ -7,6 +7,7 @@ lib_managed/ src_managed/ project/boot/ project/plugins/project/ +patch-process/* .idea .svn .classpath diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 8523333a9b3..4170bcc7def 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -3,9 +3,9 @@ * 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. @@ -26,22 +26,22 @@ import org.apache.kafka.common.utils.Utils; /** * This class is used for specifying the set of expected configurations, their type, their defaults, their * documentation, and any special validation logic used for checking the correctness of the values the user provides. - *

+ *

* Usage of this class looks something like this: - * + *

*

  * ConfigDef defs = new ConfigDef();
  * defs.define("config_name", Type.STRING, "default string value", "This configuration is used for blah blah blah.");
  * defs.define("another_config_name", Type.INT, 42, Range.atLeast(0), "More documentation on this config");
- * 
+ *
  * Properties props = new Properties();
  * props.setProperty("config_name", "some value");
  * Map<String, Object> configs = defs.parse(props);
- * 
+ *
  * String someConfig = (String) configs.get("config_name"); // will return "some value"
  * int anotherConfig = (Integer) configs.get("another_config_name"); // will return default value of 42
  * 
- * + *

* This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional * functionality for accessing configs. */ @@ -53,7 +53,7 @@ public class ConfigDef { /** * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef} - * + * * @return new unmodifiable {@link Set} instance containing the keys */ public Set names() { @@ -62,90 +62,121 @@ public class ConfigDef { /** * Define a new configuration - * - * @param name The name of the config parameter - * @param type The type of the config - * @param defaultValue The default value to use if this config isn't present - * @param validator A validator to use in checking the correctness of the config - * @param importance The importance of this config: is this something you will likely need to change. + * + * @param name The name of the config parameter + * @param type The type of the config + * @param defaultValue The default value to use if this config isn't present + * @param validator A validator to use in checking the correctness of the config + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config + * @param required Should the config fail if given property is not set and doesn't have default value specified * @return This ConfigDef so you can chain calls */ - public ConfigDef define(String name, - Type type, - Object defaultValue, - Validator validator, - Importance importance, - String documentation) { + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, + boolean required) { if (configKeys.containsKey(name)) throw new ConfigException("Configuration " + name + " is defined twice."); - Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE - : parseType(name, defaultValue, type); - configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation)); + Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type); + configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation, required)); return this; } + /** + * Define a new required configuration + * + * @param name The name of the config parameter + * @param type The type of the config + * @param defaultValue The default value to use if this config isn't present + * @param validator A validator to use in checking the correctness of the config + * @param importance The importance of this config: is this something you will likely need to change. + * @param documentation The documentation string for the config + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) { + return define(name, type, defaultValue, validator, importance, documentation, true); + } + /** * Define a new configuration with no special validation logic - * - * @param name The name of the config parameter - * @param type The type of the config - * @param defaultValue The default value to use if this config isn't present - * @param importance The importance of this config: is this something you will likely need to change. + * + * @param name The name of the config parameter + * @param type The type of the config + * @param defaultValue The default value to use if this config isn't present + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config * @return This ConfigDef so you can chain calls */ public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation) { - return define(name, type, defaultValue, null, importance, documentation); + return define(name, type, defaultValue, null, importance, documentation, true); } /** * Define a required parameter with no default value - * - * @param name The name of the config parameter - * @param type The type of the config - * @param validator A validator to use in checking the correctness of the config - * @param importance The importance of this config: is this something you will likely need to change. + * + * @param name The name of the config parameter + * @param type The type of the config + * @param validator A validator to use in checking the correctness of the config + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config * @return This ConfigDef so you can chain calls */ public ConfigDef define(String name, Type type, Validator validator, Importance importance, String documentation) { - return define(name, type, NO_DEFAULT_VALUE, validator, importance, documentation); + return define(name, type, NO_DEFAULT_VALUE, validator, importance, documentation, true); } /** * Define a required parameter with no default value and no special validation logic - * - * @param name The name of the config parameter - * @param type The type of the config - * @param importance The importance of this config: is this something you will likely need to change. + * + * @param name The name of the config parameter + * @param type The type of the config + * @param importance The importance of this config: is this something you will likely need to change. * @param documentation The documentation string for the config * @return This ConfigDef so you can chain calls */ public ConfigDef define(String name, Type type, Importance importance, String documentation) { - return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation); + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, true); } + /** + * Define a required parameter with no default value and no special validation logic + * + * @param name The name of the config parameter + * @param type The type of the config + * @param importance The importance of this config: is this something you will likely need to change. + * @param documentation The documentation string for the config + * @param required Should the config fail if given property is not set and doesn't have default value specified + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Importance importance, String documentation, boolean required) { + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, required); + } + + /** * Parse and validate configs against this configuration definition. The input is a map of configs. It is expected * that the keys of the map are strings, but the values can either be strings or they may already be of the * appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a * programmatically constructed map. - * + * * @param props The configs to parse and validate * @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into - * the appropriate type (int, string, etc) + * the appropriate type (int, string, etc) */ public Map parse(Map props) { /* parse all known keys */ Map values = new HashMap(); for (ConfigKey key : configKeys.values()) { Object value; + // props map contains setting - assign ConfigKey value if (props.containsKey(key.name)) value = parseType(key.name, props.get(key.name), key.type); - else if (key.defaultValue == NO_DEFAULT_VALUE) - throw new ConfigException("Missing required configuration \"" + key.name - + "\" which has no default value."); + // props map doesn't contain setting, the key is required and no default value specified - it's an error + else if (key.defaultValue == NO_DEFAULT_VALUE && key.required) + throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value."); + // props map doesn't contain setting, no default value specified and the key is not required - assign it to null + else if (!key.hasDefault() && !key.required) + value = null; + // otherwise assign setting it's default value else value = key.defaultValue; if (key.validator != null) @@ -157,10 +188,10 @@ public class ConfigDef { /** * Parse a value according to its expected type. - * - * @param name The config name + * + * @param name The config name * @param value The config value - * @param type The expected type + * @param type The expected type * @return The parsed object */ private Object parseType(String name, Object value, Type type) { @@ -185,8 +216,7 @@ public class ConfigDef { if (value instanceof String) return trimmed; else - throw new ConfigException(name, value, "Expected value to be a string, but it was a " - + value.getClass().getName()); + throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName()); case INT: if (value instanceof Integer) { return (Integer) value; @@ -195,6 +225,14 @@ public class ConfigDef { } else { throw new ConfigException(name, value, "Expected value to be an number."); } + case SHORT: + if (value instanceof Short) { + return (Short) value; + } else if (value instanceof String) { + return Short.parseShort(trimmed); + } else { + throw new ConfigException(name, value, "Expected value to be an number."); + } case LONG: if (value instanceof Integer) return ((Integer) value).longValue(); @@ -242,7 +280,7 @@ public class ConfigDef { * The config types */ public enum Type { - BOOLEAN, STRING, INT, LONG, DOUBLE, LIST, CLASS; + BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS; } public enum Importance { @@ -270,7 +308,7 @@ public class ConfigDef { /** * A numeric range that checks only the lower bound - * + * * @param min The minimum acceptable value */ public static Range atLeast(Number min) { @@ -303,7 +341,7 @@ public class ConfigDef { } public static class ValidString implements Validator { - private final List validStrings; + List validStrings; private ValidString(List validStrings) { this.validStrings = validStrings; @@ -316,14 +354,15 @@ public class ConfigDef { @Override public void ensureValid(String name, Object o) { String s = (String) o; - if (!validStrings.contains(s)) + if (!validStrings.contains(s)) { throw new ConfigException(name, o, "String must be one of: " + Utils.join(validStrings, ", ")); + } + } public String toString() { return "[" + Utils.join(validStrings, ", ") + "]"; } - } private static class ConfigKey { @@ -333,13 +372,9 @@ public class ConfigDef { public final Object defaultValue; public final Validator validator; public final Importance importance; + public final boolean required; - public ConfigKey(String name, - Type type, - Object defaultValue, - Validator validator, - Importance importance, - String documentation) { + public ConfigKey(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, boolean required) { super(); this.name = name; this.type = type; @@ -349,6 +384,7 @@ public class ConfigDef { if (this.validator != null) this.validator.ensureValid(name, defaultValue); this.documentation = documentation; + this.required = required; } public boolean hasDefault() { @@ -408,4 +444,4 @@ public class ConfigDef { b.append(""); return b.toString(); } -} +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/Kafka.scala b/core/src/main/scala/kafka/Kafka.scala index 77a49e12af6..37de7df0598 100644 --- a/core/src/main/scala/kafka/Kafka.scala +++ b/core/src/main/scala/kafka/Kafka.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConversions._ import joptsimple.OptionParser import metrics.KafkaMetricsReporter import server.{KafkaConfig, KafkaServerStartable, KafkaServer} -import kafka.utils.{CommandLineUtils, Utils, Logging} +import kafka.utils.{VerifiableProperties, CommandLineUtils, Utils, Logging} object Kafka extends Logging { @@ -47,13 +47,13 @@ object Kafka extends Logging { props.putAll(CommandLineUtils.parseKeyValueArgs(options.valuesOf(overrideOpt))) } - new KafkaConfig(props) + KafkaConfig.fromProps(props) } def main(args: Array[String]): Unit = { try { val serverConfig = getKafkaConfigFromArgs(args) - KafkaMetricsReporter.startReporters(serverConfig.props) + KafkaMetricsReporter.startReporters(new VerifiableProperties(serverConfig.toProps)) val kafkaServerStartable = new KafkaServerStartable(serverConfig) // attach shutdown handler to catch control-c diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index e9b4dc62df3..09fc46d759b 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1012,7 +1012,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt // if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election // is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can // eventually be restored as the leader. - if (newIsr.isEmpty && !LogConfig.fromProps(config.props.props, AdminUtils.fetchTopicConfig(zkClient, + if (newIsr.isEmpty && !LogConfig.fromProps(config.toProps, AdminUtils.fetchTopicConfig(zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition)) newIsr = leaderAndIsr.isr diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala index 4a31c7271c2..3b15ab4eef2 100644 --- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala +++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala @@ -61,7 +61,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi case true => // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration // for unclean leader election. - if (!LogConfig.fromProps(config.props.props, AdminUtils.fetchTopicConfig(controllerContext.zkClient, + if (!LogConfig.fromProps(config.toProps, AdminUtils.fetchTopicConfig(controllerContext.zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { throw new NoReplicaOnlineException(("No broker in ISR for partition " + "%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) + diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 14bf3216bae..48e33626695 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -6,7 +6,7 @@ * (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 + * 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, @@ -18,345 +18,872 @@ package kafka.server import java.util.Properties -import kafka.message.{MessageSet, Message} + import kafka.consumer.ConsumerConfig -import kafka.utils.{VerifiableProperties, ZKConfig, Utils} -import kafka.message.NoCompressionCodec -import kafka.message.BrokerCompressionCodec +import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} +import kafka.utils.Utils +import org.apache.kafka.common.config.ConfigDef -/** - * Configuration settings for the kafka server - */ -class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(props) { +import scala.collection.{JavaConversions, Map} - def this(originalProps: Properties) { - this(new VerifiableProperties(originalProps)) - props.verify() +object Defaults { + /** ********* Zookeeper Configuration ***********/ + val ZkSessionTimeoutMs = 6000 + val ZkSyncTimeMs = 2000 + + /** ********* General Configuration ***********/ + val MaxReservedBrokerId = 1000 + val BrokerId = -1 + val MessageMaxBytes = 1000000 + MessageSet.LogOverhead + val NumNetworkThreads = 3 + val NumIoThreads = 8 + val BackgroundThreads = 10 + val QueuedMaxRequests = 500 + + /** ********* Socket Server Configuration ***********/ + val Port = 9092 + val HostName: String = new String("") + val SocketSendBufferBytes: Int = 100 * 1024 + val SocketReceiveBufferBytes: Int = 100 * 1024 + val SocketRequestMaxBytes: Int = 100 * 1024 * 1024 + val MaxConnectionsPerIp: Int = Int.MaxValue + val MaxConnectionsPerIpOverrides: String = "" + val ConnectionsMaxIdleMs = 10 * 60 * 1000L + + /** ********* Log Configuration ***********/ + val NumPartitions = 1 + val LogDir = "/tmp/kafka-logs" + val LogSegmentBytes = 1 * 1024 * 1024 * 1024 + val LogRollHours = 24 * 7 + val LogRollJitterHours = 0 + val LogRetentionHours = 24 * 7 + + val LogRetentionBytes = -1L + val LogCleanupIntervalMs = 5 * 60 * 1000L + val Delete = "delete" + val Compact = "compact" + val LogCleanupPolicy = Delete + val LogCleanerThreads = 1 + val LogCleanerIoMaxBytesPerSecond = Double.MaxValue + val LogCleanerDedupeBufferSize = 500 * 1024 * 1024L + val LogCleanerIoBufferSize = 512 * 1024 + val LogCleanerDedupeBufferLoadFactor = 0.9d + val LogCleanerBackoffMs = 15 * 1000 + val LogCleanerMinCleanRatio = 0.5d + val LogCleanerEnable = false + val LogCleanerDeleteRetentionMs = 24 * 60 * 60 * 1000L + val LogIndexSizeMaxBytes = 10 * 1024 * 1024 + val LogIndexIntervalBytes = 4096 + val LogFlushIntervalMessages = Long.MaxValue + val LogDeleteDelayMs = 60000 + val LogFlushSchedulerIntervalMs = Long.MaxValue + val LogFlushOffsetCheckpointIntervalMs = 60000 + val NumRecoveryThreadsPerDataDir = 1 + val AutoCreateTopicsEnable = true + val MinInSyncReplicas = 1 + + /** ********* Replication configuration ***********/ + val ControllerSocketTimeoutMs = 30000 + val ControllerMessageQueueSize = Int.MaxValue + val DefaultReplicationFactor = 1 + val ReplicaLagTimeMaxMs = 10000L + val ReplicaLagMaxMessages = 4000 + val ReplicaSocketTimeoutMs = ConsumerConfig.SocketTimeout + val ReplicaSocketReceiveBufferBytes = ConsumerConfig.SocketBufferSize + val ReplicaFetchMaxBytes = ConsumerConfig.FetchSize + val ReplicaFetchWaitMaxMs = 500 + val ReplicaFetchMinBytes = 1 + val NumReplicaFetchers = 1 + val ReplicaHighWatermarkCheckpointIntervalMs = 5000L + val FetchPurgatoryPurgeIntervalRequests = 1000 + val ProducerPurgatoryPurgeIntervalRequests = 1000 + val AutoLeaderRebalanceEnable = true + val LeaderImbalancePerBrokerPercentage = 10 + val LeaderImbalanceCheckIntervalSeconds = 300 + val UncleanLeaderElectionEnable = true + + /** ********* Controlled shutdown configuration ***********/ + val ControlledShutdownMaxRetries = 3 + val ControlledShutdownRetryBackoffMs = 5000 + val ControlledShutdownEnable = true + + /** ********* Offset management configuration ***********/ + val OffsetMetadataMaxSize = OffsetManagerConfig.DefaultMaxMetadataSize + val OffsetsLoadBufferSize = OffsetManagerConfig.DefaultLoadBufferSize + val OffsetsTopicReplicationFactor = OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor + val OffsetsTopicPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions + val OffsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes + val OffsetsTopicCompressionCodec: Int = OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec.codec + val OffsetsRetentionMinutes: Int = 24 * 60 + val OffsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs + val OffsetCommitTimeoutMs = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs + val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks + + val DeleteTopicEnable = false + + val CompressionType = "producer" +} + +object KafkaConfig { + + /** ********* Zookeeper Configuration ***********/ + val ZkConnectProp = "zookeeper.connect" + val ZkSessionTimeoutMsProp = "zookeeper.session.timeout.ms" + val ZkConnectionTimeoutMsProp = "zookeeper.connection.timeout.ms" + val ZkSyncTimeMsProp = "zookeeper.sync.time.ms" + /** ********* General Configuration ***********/ + val MaxReservedBrokerIdProp = "reserved.broker.max.id" + val BrokerIdProp = "broker.id" + val MessageMaxBytesProp = "message.max.bytes" + val NumNetworkThreadsProp = "num.network.threads" + val NumIoThreadsProp = "num.io.threads" + val BackgroundThreadsProp = "background.threads" + val QueuedMaxRequestsProp = "queued.max.requests" + /** ********* Socket Server Configuration ***********/ + val PortProp = "port" + val HostNameProp = "host.name" + val AdvertisedHostNameProp: String = "advertised.host.name" + val AdvertisedPortProp = "advertised.port" + val SocketSendBufferBytesProp = "socket.send.buffer.bytes" + val SocketReceiveBufferBytesProp = "socket.receive.buffer.bytes" + val SocketRequestMaxBytesProp = "socket.request.max.bytes" + val MaxConnectionsPerIpProp = "max.connections.per.ip" + val MaxConnectionsPerIpOverridesProp = "max.connections.per.ip.overrides" + val ConnectionsMaxIdleMsProp = "connections.max.idle.ms" + /** ********* Log Configuration ***********/ + val NumPartitionsProp = "num.partitions" + val LogDirsProp = "log.dirs" + val LogDirProp = "log.dir" + val LogSegmentBytesProp = "log.segment.bytes" + + val LogRollTimeMillisProp = "log.roll.ms" + val LogRollTimeHoursProp = "log.roll.hours" + + val LogRollTimeJitterMillisProp = "log.roll.jitter.ms" + val LogRollTimeJitterHoursProp = "log.roll.jitter.hours" + + val LogRetentionTimeMillisProp = "log.retention.ms" + val LogRetentionTimeMinutesProp = "log.retention.minutes" + val LogRetentionTimeHoursProp = "log.retention.hours" + + val LogRetentionBytesProp = "log.retention.bytes" + val LogCleanupIntervalMsProp = "log.retention.check.interval.ms" + val LogCleanupPolicyProp = "log.cleanup.policy" + val LogCleanerThreadsProp = "log.cleaner.threads" + val LogCleanerIoMaxBytesPerSecondProp = "log.cleaner.io.max.bytes.per.second" + val LogCleanerDedupeBufferSizeProp = "log.cleaner.dedupe.buffer.size" + val LogCleanerIoBufferSizeProp = "log.cleaner.io.buffer.size" + val LogCleanerDedupeBufferLoadFactorProp = "log.cleaner.io.buffer.load.factor" + val LogCleanerBackoffMsProp = "log.cleaner.backoff.ms" + val LogCleanerMinCleanRatioProp = "log.cleaner.min.cleanable.ratio" + val LogCleanerEnableProp = "log.cleaner.enable" + val LogCleanerDeleteRetentionMsProp = "log.cleaner.delete.retention.ms" + val LogIndexSizeMaxBytesProp = "log.index.size.max.bytes" + val LogIndexIntervalBytesProp = "log.index.interval.bytes" + val LogFlushIntervalMessagesProp = "log.flush.interval.messages" + val LogDeleteDelayMsProp = "log.segment.delete.delay.ms" + val LogFlushSchedulerIntervalMsProp = "log.flush.scheduler.interval.ms" + val LogFlushIntervalMsProp = "log.flush.interval.ms" + val LogFlushOffsetCheckpointIntervalMsProp = "log.flush.offset.checkpoint.interval.ms" + val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" + val AutoCreateTopicsEnableProp = "auto.create.topics.enable" + val MinInSyncReplicasProp = "min.insync.replicas" + /** ********* Replication configuration ***********/ + val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" + val ControllerMessageQueueSizeProp = "controller.message.queue.size" + val DefaultReplicationFactorProp = "default.replication.factor" + val ReplicaLagTimeMaxMsProp = "replica.lag.time.max.ms" + val ReplicaLagMaxMessagesProp = "replica.lag.max.messages" + val ReplicaSocketTimeoutMsProp = "replica.socket.timeout.ms" + val ReplicaSocketReceiveBufferBytesProp = "replica.socket.receive.buffer.bytes" + val ReplicaFetchMaxBytesProp = "replica.fetch.max.bytes" + val ReplicaFetchWaitMaxMsProp = "replica.fetch.wait.max.ms" + val ReplicaFetchMinBytesProp = "replica.fetch.min.bytes" + val NumReplicaFetchersProp = "num.replica.fetchers" + val ReplicaHighWatermarkCheckpointIntervalMsProp = "replica.high.watermark.checkpoint.interval.ms" + val FetchPurgatoryPurgeIntervalRequestsProp = "fetch.purgatory.purge.interval.requests" + val ProducerPurgatoryPurgeIntervalRequestsProp = "producer.purgatory.purge.interval.requests" + val AutoLeaderRebalanceEnableProp = "auto.leader.rebalance.enable" + val LeaderImbalancePerBrokerPercentageProp = "leader.imbalance.per.broker.percentage" + val LeaderImbalanceCheckIntervalSecondsProp = "leader.imbalance.check.interval.seconds" + val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" + /** ********* Controlled shutdown configuration ***********/ + val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" + val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" + val ControlledShutdownEnableProp = "controlled.shutdown.enable" + /** ********* Offset management configuration ***********/ + val OffsetMetadataMaxSizeProp = "offset.metadata.max.bytes" + val OffsetsLoadBufferSizeProp = "offsets.load.buffer.size" + val OffsetsTopicReplicationFactorProp = "offsets.topic.replication.factor" + val OffsetsTopicPartitionsProp = "offsets.topic.num.partitions" + val OffsetsTopicSegmentBytesProp = "offsets.topic.segment.bytes" + val OffsetsTopicCompressionCodecProp = "offsets.topic.compression.codec" + val OffsetsRetentionMinutesProp = "offsets.retention.minutes" + val OffsetsRetentionCheckIntervalMsProp = "offsets.retention.check.interval.ms" + val OffsetCommitTimeoutMsProp = "offsets.commit.timeout.ms" + val OffsetCommitRequiredAcksProp = "offsets.commit.required.acks" + + val DeleteTopicEnableProp = "delete.topic.enable" + val CompressionTypeProp = "compression.type" + + + /* Documentation */ + /** ********* Zookeeper Configuration ***********/ + val ZkConnectDoc = "Zookeeper host string" + val ZkSessionTimeoutMsDoc = "Zookeeper session timeout" + val ZkConnectionTimeoutMsDoc = "The max time that the client waits to establish a connection to zookeeper" + val ZkSyncTimeMsDoc = "How far a ZK follower can be behind a ZK leader" + /** ********* General Configuration ***********/ + val MaxReservedBrokerIdDoc = "Max number that can be used for a broker.id" + val BrokerIdDoc = "The broker id for this server. " + + "To avoid conflicts between zookeeper generated brokerId and user's config.brokerId " + + "added MaxReservedBrokerId and zookeeper sequence starts from MaxReservedBrokerId + 1." + val MessageMaxBytesDoc = "The maximum size of message that the server can receive" + val NumNetworkThreadsDoc = "the number of network threads that the server uses for handling network requests" + val NumIoThreadsDoc = "The number of io threads that the server uses for carrying out network requests" + val BackgroundThreadsDoc = "The number of threads to use for various background processing tasks" + val QueuedMaxRequestsDoc = "The number of queued requests allowed before blocking the network threads" + /** ********* Socket Server Configuration ***********/ + val PortDoc = "the port to listen and accept connections on" + val HostNameDoc = "hostname of broker. If this is set, it will only bind to this address. If this is not set, it will bind to all interfaces" + val AdvertisedHostNameDoc = "Hostname to publish to ZooKeeper for clients to use. In IaaS environments, this may " + + "need to be different from the interface to which the broker binds. If this is not set, " + + "it will use the value for \"host.name\" if configured. Otherwise " + + "it will use the value returned from java.net.InetAddress.getCanonicalHostName()." + val AdvertisedPortDoc = "The port to publish to ZooKeeper for clients to use. In IaaS environments, this may " + + "need to be different from the port to which the broker binds. If this is not set, " + + "it will publish the same port that the broker binds to." + val SocketSendBufferBytesDoc = "The SO_SNDBUF buffer of the socket sever sockets" + val SocketReceiveBufferBytesDoc = "The SO_RCVBUF buffer of the socket sever sockets" + val SocketRequestMaxBytesDoc = "The maximum number of bytes in a socket request" + val MaxConnectionsPerIpDoc = "The maximum number of connections we allow from each ip address" + val MaxConnectionsPerIpOverridesDoc = "Per-ip or hostname overrides to the default maximum number of connections" + val ConnectionsMaxIdleMsDoc = "Idle connections timeout: the server socket processor threads close the connections that idle more than this" + /** ********* 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 = "The directories in which the log data is kept" + 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)" + 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)" + 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)" + val LogRetentionTimeMinsDoc = "The number of minutes to keep a log file before deleting it (in minutes), secondary to " + LogRetentionTimeMillisProp + " property" + 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 minutes that the log cleaner checks whether any log is eligible for deletion" + val LogCleanupPolicyDoc = "The default cleanup policy for segments beyond the retention window, must be either \"delete\" or \"compact\"" + val LogCleanerThreadsDoc = "The number of background threads to use for log cleaning" + val LogCleanerIoMaxBytesPerSecondDoc = "The log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average" + val LogCleanerDedupeBufferSizeDoc = "The total memory used for log deduplication across all cleaner threads" + val LogCleanerIoBufferSizeDoc = "The total memory used for log cleaner I/O buffers across all cleaner threads" + val LogCleanerDedupeBufferLoadFactorDoc = "Log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value " + + "will allow more log to be cleaned at once but will lead to more hash collisions" + val LogCleanerBackoffMsDoc = "The amount of time to sleep when there are no logs to clean" + val LogCleanerMinCleanRatioDoc = "The minimum ratio of dirty log to total log for a log to eligible for cleaning" + val LogCleanerEnableDoc = "Should we enable log cleaning?" + val LogCleanerDeleteRetentionMsDoc = "How long are delete records retained?" + 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" + val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point" + 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 = "define the minimum number of replicas in ISR needed to satisfy a produce request with required.acks=-1 (or all)" + /** ********* Replication configuration ***********/ + val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels" + val ControllerMessageQueueSizeDoc = "The buffer size for controller-to-broker-channels" + val DefaultReplicationFactorDoc = "default replication factors for automatically created topics" + val ReplicaLagTimeMaxMsDoc = "If a follower hasn't sent any fetch requests during this time, the leader will remove the follower from isr" + val ReplicaLagMaxMessagesDoc = "If the lag in messages between a leader and a follower exceeds this number, the leader will remove the follower from isr" + val ReplicaSocketTimeoutMsDoc = "The socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms" + val ReplicaSocketReceiveBufferBytesDoc = "The socket receive buffer for network requests" + val ReplicaFetchMaxBytesDoc = "The number of byes of messages to attempt to fetch" + val ReplicaFetchWaitMaxMsDoc = "max wait time for each fetcher request issued by follower replicas. This value should always be less than the " + + "replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics" + val ReplicaFetchMinBytesDoc = "Minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs" + val NumReplicaFetchersDoc = "Number of fetcher threads used to replicate messages from a source broker. " + + "Increasing this value can increase the degree of I/O parallelism in the follower broker." + val ReplicaHighWatermarkCheckpointIntervalMsDoc = "The frequency with which the high watermark is saved out to disk" + val FetchPurgatoryPurgeIntervalRequestsDoc = "The purge interval (in number of requests) of the fetch request purgatory" + val ProducerPurgatoryPurgeIntervalRequestsDoc = "The purge interval (in number of requests) of the producer request purgatory" + val AutoLeaderRebalanceEnableDoc = "Enables auto leader balancing. A background thread checks and triggers leader balance if required at regular intervals" + val LeaderImbalancePerBrokerPercentageDoc = "The ratio of leader imbalance allowed per broker. The controller would trigger a leader balance if it goes above this value per broker. The value is specified in percentage." + val LeaderImbalanceCheckIntervalSecondsDoc = "The frequency with which the partition rebalance check is triggered by the controller" + val UncleanLeaderElectionEnableDoc = "Indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though doing so may result in data loss" + /** ********* Controlled shutdown configuration ***********/ + val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" + val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." + val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server" + /** ********* Offset management configuration ***********/ + val OffsetMetadataMaxSizeDoc = "The maximum size for a metadata entry associated with an offset commit" + val OffsetsLoadBufferSizeDoc = "Batch size for reading from the offsets segments when loading offsets into the cache." + val OffsetsTopicReplicationFactorDoc = "The replication factor for the offsets topic (set higher to ensure availability). " + + "To ensure that the effective replication factor of the offsets topic is the configured value, " + + "the number of alive brokers has to be at least the replication factor at the time of the " + + "first request for the offsets topic. If not, either the offsets topic creation will fail or " + + "it will get a replication factor of min(alive brokers, configured replication factor)" + val OffsetsTopicPartitionsDoc = "The number of partitions for the offset commit topic (should not change after deployment)" + val OffsetsTopicSegmentBytesDoc = "The offsets topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads" + val OffsetsTopicCompressionCodecDoc = "Compression codec for the offsets topic - compression may be used to achieve \"atomic\" commits" + val OffsetsRetentionMinutesDoc = "Offsets older than this retention period will be discarded" + val OffsetsRetentionCheckIntervalMsDoc = "Frequency at which to check for stale offsets" + val OffsetCommitTimeoutMsDoc = "Offset commit will be delayed until all replicas for the offsets topic receive the commit " + + "or this timeout is reached. This is similar to the producer request timeout." + val OffsetCommitRequiredAcksDoc = "The required acks before the commit can be accepted. In general, the default (-1) should not be overridden" + val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" + val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + + "('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + + "'producer' which means retain the original compression codec set by the producer." + + + private val configDef = { + import ConfigDef.Range._ + import ConfigDef.ValidString._ + import ConfigDef.Type._ + import ConfigDef.Importance._ + import java.util.Arrays.asList + + new ConfigDef() + + /** ********* Zookeeper Configuration ***********/ + .define(ZkConnectProp, STRING, HIGH, ZkConnectDoc) + .define(ZkSessionTimeoutMsProp, INT, Defaults.ZkSessionTimeoutMs, HIGH, ZkSessionTimeoutMsDoc) + .define(ZkConnectionTimeoutMsProp, INT, HIGH, ZkConnectionTimeoutMsDoc, false) + .define(ZkSyncTimeMsProp, INT, Defaults.ZkSyncTimeMs, LOW, ZkSyncTimeMsDoc) + + /** ********* General Configuration ***********/ + .define(MaxReservedBrokerIdProp, INT, Defaults.MaxReservedBrokerId, atLeast(0), MEDIUM, MaxReservedBrokerIdProp) + .define(BrokerIdProp, INT, Defaults.BrokerId, HIGH, BrokerIdDoc) + .define(MessageMaxBytesProp, INT, Defaults.MessageMaxBytes, atLeast(0), HIGH, MessageMaxBytesDoc) + .define(NumNetworkThreadsProp, INT, Defaults.NumNetworkThreads, atLeast(1), HIGH, NumNetworkThreadsDoc) + .define(NumIoThreadsProp, INT, Defaults.NumIoThreads, atLeast(1), HIGH, NumIoThreadsDoc) + .define(BackgroundThreadsProp, INT, Defaults.BackgroundThreads, atLeast(1), HIGH, BackgroundThreadsDoc) + .define(QueuedMaxRequestsProp, INT, Defaults.QueuedMaxRequests, atLeast(1), HIGH, QueuedMaxRequestsDoc) + + /** ********* Socket Server Configuration ***********/ + .define(PortProp, INT, Defaults.Port, HIGH, PortDoc) + .define(HostNameProp, STRING, Defaults.HostName, HIGH, HostNameDoc) + .define(AdvertisedHostNameProp, STRING, HIGH, AdvertisedHostNameDoc, false) + .define(AdvertisedPortProp, INT, HIGH, AdvertisedPortDoc, false) + .define(SocketSendBufferBytesProp, INT, Defaults.SocketSendBufferBytes, HIGH, SocketSendBufferBytesDoc) + .define(SocketReceiveBufferBytesProp, INT, Defaults.SocketReceiveBufferBytes, HIGH, SocketReceiveBufferBytesDoc) + .define(SocketRequestMaxBytesProp, INT, Defaults.SocketRequestMaxBytes, atLeast(1), HIGH, SocketRequestMaxBytesDoc) + .define(MaxConnectionsPerIpProp, INT, Defaults.MaxConnectionsPerIp, atLeast(1), MEDIUM, MaxConnectionsPerIpDoc) + .define(MaxConnectionsPerIpOverridesProp, STRING, Defaults.MaxConnectionsPerIpOverrides, MEDIUM, MaxConnectionsPerIpOverridesDoc) + .define(ConnectionsMaxIdleMsProp, LONG, Defaults.ConnectionsMaxIdleMs, MEDIUM, ConnectionsMaxIdleMsDoc) + + /** ********* Log Configuration ***********/ + .define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc) + .define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc) + .define(LogDirsProp, STRING, HIGH, LogDirsDoc, false) + .define(LogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(Message.MinHeaderSize), HIGH, LogSegmentBytesDoc) + + .define(LogRollTimeMillisProp, LONG, HIGH, LogRollTimeMillisDoc, false) + .define(LogRollTimeHoursProp, INT, Defaults.LogRollHours, atLeast(1), HIGH, LogRollTimeHoursDoc) + + .define(LogRollTimeJitterMillisProp, LONG, HIGH, LogRollTimeJitterMillisDoc, false) + .define(LogRollTimeJitterHoursProp, INT, Defaults.LogRollJitterHours, atLeast(0), HIGH, LogRollTimeJitterHoursDoc) + + .define(LogRetentionTimeMillisProp, LONG, HIGH, LogRetentionTimeMillisDoc, false) + .define(LogRetentionTimeMinutesProp, INT, HIGH, LogRetentionTimeMinsDoc, false) + .define(LogRetentionTimeHoursProp, INT, Defaults.LogRetentionHours, atLeast(1), HIGH, LogRetentionTimeHoursDoc) + + .define(LogRetentionBytesProp, LONG, Defaults.LogRetentionBytes, HIGH, LogRetentionBytesDoc) + .define(LogCleanupIntervalMsProp, LONG, Defaults.LogCleanupIntervalMs, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc) + .define(LogCleanupPolicyProp, STRING, Defaults.LogCleanupPolicy, in(Defaults.Compact, Defaults.Delete), MEDIUM, LogCleanupPolicyDoc) + .define(LogCleanerThreadsProp, INT, Defaults.LogCleanerThreads, atLeast(0), MEDIUM, LogCleanerThreadsDoc) + .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LogCleanerIoMaxBytesPerSecond, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc) + .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LogCleanerDedupeBufferSize, MEDIUM, LogCleanerDedupeBufferSizeDoc) + .define(LogCleanerIoBufferSizeProp, INT, Defaults.LogCleanerIoBufferSize, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) + .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LogCleanerDedupeBufferLoadFactor, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) + .define(LogCleanerBackoffMsProp, LONG, Defaults.LogCleanerBackoffMs, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) + .define(LogCleanerMinCleanRatioProp, DOUBLE, Defaults.LogCleanerMinCleanRatio, MEDIUM, LogCleanerMinCleanRatioDoc) + .define(LogCleanerEnableProp, BOOLEAN, Defaults.LogCleanerEnable, MEDIUM, LogCleanerEnableDoc) + .define(LogCleanerDeleteRetentionMsProp, LONG, Defaults.LogCleanerDeleteRetentionMs, MEDIUM, LogCleanerDeleteRetentionMsDoc) + .define(LogIndexSizeMaxBytesProp, INT, Defaults.LogIndexSizeMaxBytes, atLeast(4), MEDIUM, LogIndexSizeMaxBytesDoc) + .define(LogIndexIntervalBytesProp, INT, Defaults.LogIndexIntervalBytes, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc) + .define(LogFlushIntervalMessagesProp, LONG, Defaults.LogFlushIntervalMessages, atLeast(1), HIGH, LogFlushIntervalMessagesDoc) + .define(LogDeleteDelayMsProp, LONG, Defaults.LogDeleteDelayMs, atLeast(0), HIGH, LogDeleteDelayMsDoc) + .define(LogFlushSchedulerIntervalMsProp, LONG, Defaults.LogFlushSchedulerIntervalMs, HIGH, LogFlushSchedulerIntervalMsDoc) + .define(LogFlushIntervalMsProp, LONG, HIGH, LogFlushIntervalMsDoc, false) + .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) + .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) + .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) + .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) + + /** ********* Replication configuration ***********/ + .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) + .define(ControllerMessageQueueSizeProp, INT, Defaults.ControllerMessageQueueSize, MEDIUM, ControllerMessageQueueSizeDoc) + .define(DefaultReplicationFactorProp, INT, Defaults.DefaultReplicationFactor, MEDIUM, DefaultReplicationFactorDoc) + .define(ReplicaLagTimeMaxMsProp, LONG, Defaults.ReplicaLagTimeMaxMs, HIGH, ReplicaLagTimeMaxMsDoc) + .define(ReplicaLagMaxMessagesProp, LONG, Defaults.ReplicaLagMaxMessages, HIGH, ReplicaLagMaxMessagesDoc) + .define(ReplicaSocketTimeoutMsProp, INT, Defaults.ReplicaSocketTimeoutMs, HIGH, ReplicaSocketTimeoutMsDoc) + .define(ReplicaSocketReceiveBufferBytesProp, INT, Defaults.ReplicaSocketReceiveBufferBytes, HIGH, ReplicaSocketReceiveBufferBytesDoc) + .define(ReplicaFetchMaxBytesProp, INT, Defaults.ReplicaFetchMaxBytes, HIGH, ReplicaFetchMaxBytesDoc) + .define(ReplicaFetchWaitMaxMsProp, INT, Defaults.ReplicaFetchWaitMaxMs, HIGH, ReplicaFetchWaitMaxMsDoc) + .define(ReplicaFetchMinBytesProp, INT, Defaults.ReplicaFetchMinBytes, HIGH, ReplicaFetchMinBytesDoc) + .define(NumReplicaFetchersProp, INT, Defaults.NumReplicaFetchers, HIGH, NumReplicaFetchersDoc) + .define(ReplicaHighWatermarkCheckpointIntervalMsProp, LONG, Defaults.ReplicaHighWatermarkCheckpointIntervalMs, HIGH, ReplicaHighWatermarkCheckpointIntervalMsDoc) + .define(FetchPurgatoryPurgeIntervalRequestsProp, INT, Defaults.FetchPurgatoryPurgeIntervalRequests, MEDIUM, FetchPurgatoryPurgeIntervalRequestsDoc) + .define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.ProducerPurgatoryPurgeIntervalRequests, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc) + .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) + .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) + .define(LeaderImbalanceCheckIntervalSecondsProp, INT, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) + .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) + + /** ********* Controlled shutdown configuration ***********/ + .define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) + .define(ControlledShutdownRetryBackoffMsProp, INT, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) + .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) + + /** ********* Offset management configuration ***********/ + .define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc) + .define(OffsetsLoadBufferSizeProp, INT, Defaults.OffsetsLoadBufferSize, atLeast(1), HIGH, OffsetsLoadBufferSizeDoc) + .define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.OffsetsTopicReplicationFactor, atLeast(1), HIGH, OffsetsTopicReplicationFactorDoc) + .define(OffsetsTopicPartitionsProp, INT, Defaults.OffsetsTopicPartitions, atLeast(1), HIGH, OffsetsTopicPartitionsDoc) + .define(OffsetsTopicSegmentBytesProp, INT, Defaults.OffsetsTopicSegmentBytes, atLeast(1), HIGH, OffsetsTopicSegmentBytesDoc) + .define(OffsetsTopicCompressionCodecProp, INT, Defaults.OffsetsTopicCompressionCodec, HIGH, OffsetsTopicCompressionCodecDoc) + .define(OffsetsRetentionMinutesProp, INT, Defaults.OffsetsRetentionMinutes, atLeast(1), HIGH, OffsetsRetentionMinutesDoc) + .define(OffsetsRetentionCheckIntervalMsProp, LONG, Defaults.OffsetsRetentionCheckIntervalMs, atLeast(1), HIGH, OffsetsRetentionCheckIntervalMsDoc) + .define(OffsetCommitTimeoutMsProp, INT, Defaults.OffsetCommitTimeoutMs, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc) + .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) + .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) + .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) } - private def getLogRetentionTimeMillis(): Long = { + def configNames() = { + import scala.collection.JavaConversions._ + configDef.names().toList.sorted + } + + /** + * Parse the given properties instance into a KafkaConfig object + */ + def fromProps(props: Properties): KafkaConfig = { + import kafka.utils.Utils.evaluateDefaults + val parsed = configDef.parse(evaluateDefaults(props)) + new KafkaConfig( + /** ********* Zookeeper Configuration ***********/ + zkConnect = parsed.get(ZkConnectProp).asInstanceOf[String], + zkSessionTimeoutMs = parsed.get(ZkSessionTimeoutMsProp).asInstanceOf[Int], + _zkConnectionTimeoutMs = Option(parsed.get(ZkConnectionTimeoutMsProp)).map(_.asInstanceOf[Int]), + zkSyncTimeMs = parsed.get(ZkSyncTimeMsProp).asInstanceOf[Int], + + /** ********* General Configuration ***********/ + maxReservedBrokerId = parsed.get(MaxReservedBrokerIdProp).asInstanceOf[Int], + brokerId = parsed.get(BrokerIdProp).asInstanceOf[Int], + messageMaxBytes = parsed.get(MessageMaxBytesProp).asInstanceOf[Int], + numNetworkThreads = parsed.get(NumNetworkThreadsProp).asInstanceOf[Int], + numIoThreads = parsed.get(NumIoThreadsProp).asInstanceOf[Int], + backgroundThreads = parsed.get(BackgroundThreadsProp).asInstanceOf[Int], + queuedMaxRequests = parsed.get(QueuedMaxRequestsProp).asInstanceOf[Int], + + /** ********* Socket Server Configuration ***********/ + port = parsed.get(PortProp).asInstanceOf[Int], + hostName = parsed.get(HostNameProp).asInstanceOf[String], + _advertisedHostName = Option(parsed.get(AdvertisedHostNameProp)).map(_.asInstanceOf[String]), + _advertisedPort = Option(parsed.get(AdvertisedPortProp)).map(_.asInstanceOf[Int]), + socketSendBufferBytes = parsed.get(SocketSendBufferBytesProp).asInstanceOf[Int], + socketReceiveBufferBytes = parsed.get(SocketReceiveBufferBytesProp).asInstanceOf[Int], + socketRequestMaxBytes = parsed.get(SocketRequestMaxBytesProp).asInstanceOf[Int], + maxConnectionsPerIp = parsed.get(MaxConnectionsPerIpProp).asInstanceOf[Int], + _maxConnectionsPerIpOverrides = parsed.get(MaxConnectionsPerIpOverridesProp).asInstanceOf[String], + connectionsMaxIdleMs = parsed.get(ConnectionsMaxIdleMsProp).asInstanceOf[Long], + + /** ********* Log Configuration ***********/ + numPartitions = parsed.get(NumPartitionsProp).asInstanceOf[Int], + _logDir = parsed.get(LogDirProp).asInstanceOf[String], + _logDirs = Option(parsed.get(LogDirsProp)).map(_.asInstanceOf[String]), + + logSegmentBytes = parsed.get(LogSegmentBytesProp).asInstanceOf[Int], + logRollTimeHours = parsed.get(LogRollTimeHoursProp).asInstanceOf[Int], + _logRollTimeMillis = Option(parsed.get(LogRollTimeMillisProp)).map(_.asInstanceOf[Long]), + + logRollTimeJitterHours = parsed.get(LogRollTimeJitterHoursProp).asInstanceOf[Int], + _logRollTimeJitterMillis = Option(parsed.get(LogRollTimeJitterMillisProp)).map(_.asInstanceOf[Long]), + + logRetentionTimeHours = parsed.get(LogRetentionTimeHoursProp).asInstanceOf[Int], + _logRetentionTimeMins = Option(parsed.get(LogRetentionTimeMinutesProp)).map(_.asInstanceOf[Int]), + _logRetentionTimeMillis = Option(parsed.get(LogRetentionTimeMillisProp)).map(_.asInstanceOf[Long]), + + logRetentionBytes = parsed.get(LogRetentionBytesProp).asInstanceOf[Long], + logCleanupIntervalMs = parsed.get(LogCleanupIntervalMsProp).asInstanceOf[Long], + logCleanupPolicy = parsed.get(LogCleanupPolicyProp).asInstanceOf[String], + logCleanerThreads = parsed.get(LogCleanerThreadsProp).asInstanceOf[Int], + logCleanerIoMaxBytesPerSecond = parsed.get(LogCleanerIoMaxBytesPerSecondProp).asInstanceOf[Double], + logCleanerDedupeBufferSize = parsed.get(LogCleanerDedupeBufferSizeProp).asInstanceOf[Long], + logCleanerIoBufferSize = parsed.get(LogCleanerIoBufferSizeProp).asInstanceOf[Int], + logCleanerDedupeBufferLoadFactor = parsed.get(LogCleanerDedupeBufferLoadFactorProp).asInstanceOf[Double], + logCleanerBackoffMs = parsed.get(LogCleanerBackoffMsProp).asInstanceOf[Long], + logCleanerMinCleanRatio = parsed.get(LogCleanerMinCleanRatioProp).asInstanceOf[Double], + logCleanerEnable = parsed.get(LogCleanerEnableProp).asInstanceOf[Boolean], + logCleanerDeleteRetentionMs = parsed.get(LogCleanerDeleteRetentionMsProp).asInstanceOf[Long], + logIndexSizeMaxBytes = parsed.get(LogIndexSizeMaxBytesProp).asInstanceOf[Int], + logIndexIntervalBytes = parsed.get(LogIndexIntervalBytesProp).asInstanceOf[Int], + logFlushIntervalMessages = parsed.get(LogFlushIntervalMessagesProp).asInstanceOf[Long], + logDeleteDelayMs = parsed.get(LogDeleteDelayMsProp).asInstanceOf[Long], + logFlushSchedulerIntervalMs = parsed.get(LogFlushSchedulerIntervalMsProp).asInstanceOf[Long], + _logFlushIntervalMs = Option(parsed.get(LogFlushIntervalMsProp)).map(_.asInstanceOf[Long]), + logFlushOffsetCheckpointIntervalMs = parsed.get(LogFlushOffsetCheckpointIntervalMsProp).asInstanceOf[Int], + numRecoveryThreadsPerDataDir = parsed.get(NumRecoveryThreadsPerDataDirProp).asInstanceOf[Int], + autoCreateTopicsEnable = parsed.get(AutoCreateTopicsEnableProp).asInstanceOf[Boolean], + minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int], + + /** ********* Replication configuration ***********/ + controllerSocketTimeoutMs = parsed.get(ControllerSocketTimeoutMsProp).asInstanceOf[Int], + controllerMessageQueueSize = parsed.get(ControllerMessageQueueSizeProp).asInstanceOf[Int], + defaultReplicationFactor = parsed.get(DefaultReplicationFactorProp).asInstanceOf[Int], + replicaLagTimeMaxMs = parsed.get(ReplicaLagTimeMaxMsProp).asInstanceOf[Long], + replicaLagMaxMessages = parsed.get(ReplicaLagMaxMessagesProp).asInstanceOf[Long], + replicaSocketTimeoutMs = parsed.get(ReplicaSocketTimeoutMsProp).asInstanceOf[Int], + replicaSocketReceiveBufferBytes = parsed.get(ReplicaSocketReceiveBufferBytesProp).asInstanceOf[Int], + replicaFetchMaxBytes = parsed.get(ReplicaFetchMaxBytesProp).asInstanceOf[Int], + replicaFetchWaitMaxMs = parsed.get(ReplicaFetchWaitMaxMsProp).asInstanceOf[Int], + replicaFetchMinBytes = parsed.get(ReplicaFetchMinBytesProp).asInstanceOf[Int], + numReplicaFetchers = parsed.get(NumReplicaFetchersProp).asInstanceOf[Int], + replicaHighWatermarkCheckpointIntervalMs = parsed.get(ReplicaHighWatermarkCheckpointIntervalMsProp).asInstanceOf[Long], + fetchPurgatoryPurgeIntervalRequests = parsed.get(FetchPurgatoryPurgeIntervalRequestsProp).asInstanceOf[Int], + producerPurgatoryPurgeIntervalRequests = parsed.get(ProducerPurgatoryPurgeIntervalRequestsProp).asInstanceOf[Int], + autoLeaderRebalanceEnable = parsed.get(AutoLeaderRebalanceEnableProp).asInstanceOf[Boolean], + leaderImbalancePerBrokerPercentage = parsed.get(LeaderImbalancePerBrokerPercentageProp).asInstanceOf[Int], + leaderImbalanceCheckIntervalSeconds = parsed.get(LeaderImbalanceCheckIntervalSecondsProp).asInstanceOf[Int], + uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], + + /** ********* Controlled shutdown configuration ***********/ + controlledShutdownMaxRetries = parsed.get(ControlledShutdownMaxRetriesProp).asInstanceOf[Int], + controlledShutdownRetryBackoffMs = parsed.get(ControlledShutdownRetryBackoffMsProp).asInstanceOf[Int], + controlledShutdownEnable = parsed.get(ControlledShutdownEnableProp).asInstanceOf[Boolean], + + /** ********* Offset management configuration ***********/ + offsetMetadataMaxSize = parsed.get(OffsetMetadataMaxSizeProp).asInstanceOf[Int], + offsetsLoadBufferSize = parsed.get(OffsetsLoadBufferSizeProp).asInstanceOf[Int], + offsetsTopicReplicationFactor = parsed.get(OffsetsTopicReplicationFactorProp).asInstanceOf[Short], + offsetsTopicPartitions = parsed.get(OffsetsTopicPartitionsProp).asInstanceOf[Int], + offsetsTopicSegmentBytes = parsed.get(OffsetsTopicSegmentBytesProp).asInstanceOf[Int], + offsetsTopicCompressionCodec = Option(parsed.get(OffsetsTopicCompressionCodecProp)).map(_.asInstanceOf[Int]).map(value => CompressionCodec.getCompressionCodec(value)).orNull, + offsetsRetentionMinutes = parsed.get(OffsetsRetentionMinutesProp).asInstanceOf[Int], + offsetsRetentionCheckIntervalMs = parsed.get(OffsetsRetentionCheckIntervalMsProp).asInstanceOf[Long], + offsetCommitTimeoutMs = parsed.get(OffsetCommitTimeoutMsProp).asInstanceOf[Int], + offsetCommitRequiredAcks = parsed.get(OffsetCommitRequiredAcksProp).asInstanceOf[Short], + deleteTopicEnable = parsed.get(DeleteTopicEnableProp).asInstanceOf[Boolean], + compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String] + ) + } + + /** + * Create a log config instance using the given properties and defaults + */ + def fromProps(defaults: Properties, overrides: Properties): KafkaConfig = { + val props = new Properties(defaults) + props.putAll(overrides) + fromProps(props) + } + + /** + * Check that property names are valid + */ + def validateNames(props: Properties) { + import JavaConversions._ + val names = configDef.names() + for (name <- props.keys) + require(names.contains(name), "Unknown configuration \"%s\".".format(name)) + } + + /** + * Check that the given properties contain only valid kafka config names and that all values can be parsed and are valid + */ + def validate(props: Properties) { + validateNames(props) + configDef.parse(props) + + // to bootstrap KafkaConfig.validateValues() + KafkaConfig.fromProps(props) + } +} + +class KafkaConfig(/** ********* Zookeeper Configuration ***********/ + val zkConnect: String, + val zkSessionTimeoutMs: Int = Defaults.ZkSessionTimeoutMs, + private val _zkConnectionTimeoutMs: Option[Int] = None, + val zkSyncTimeMs: Int = Defaults.ZkSyncTimeMs, + + /** ********* General Configuration ***********/ + val maxReservedBrokerId: Int = Defaults.MaxReservedBrokerId, + var brokerId: Int = Defaults.BrokerId, + val messageMaxBytes: Int = Defaults.MessageMaxBytes, + val numNetworkThreads: Int = Defaults.NumNetworkThreads, + val numIoThreads: Int = Defaults.NumIoThreads, + val backgroundThreads: Int = Defaults.BackgroundThreads, + val queuedMaxRequests: Int = Defaults.QueuedMaxRequests, + + /** ********* Socket Server Configuration ***********/ + val port: Int = Defaults.Port, + val hostName: String = Defaults.HostName, + private val _advertisedHostName: Option[String] = None, + private val _advertisedPort: Option[Int] = None, + val socketSendBufferBytes: Int = Defaults.SocketSendBufferBytes, + val socketReceiveBufferBytes: Int = Defaults.SocketReceiveBufferBytes, + val socketRequestMaxBytes: Int = Defaults.SocketRequestMaxBytes, + val maxConnectionsPerIp: Int = Defaults.MaxConnectionsPerIp, + private val _maxConnectionsPerIpOverrides: String = Defaults.MaxConnectionsPerIpOverrides, + val connectionsMaxIdleMs: Long = Defaults.ConnectionsMaxIdleMs, + + /** ********* Log Configuration ***********/ + val numPartitions: Int = Defaults.NumPartitions, + private val _logDir: String = Defaults.LogDir, + private val _logDirs: Option[String] = None, + + val logSegmentBytes: Int = Defaults.LogSegmentBytes, + + val logRollTimeHours: Int = Defaults.LogRollHours, + private val _logRollTimeMillis: Option[Long] = None, + + val logRollTimeJitterHours: Int = Defaults.LogRollJitterHours, + private val _logRollTimeJitterMillis: Option[Long] = None, + + val logRetentionTimeHours: Int = Defaults.LogRetentionHours, + private val _logRetentionTimeMins: Option[Int] = None, + private val _logRetentionTimeMillis: Option[Long] = None, + + val logRetentionBytes: Long = Defaults.LogRetentionBytes, + val logCleanupIntervalMs: Long = Defaults.LogCleanupIntervalMs, + val logCleanupPolicy: String = Defaults.LogCleanupPolicy, + val logCleanerThreads: Int = Defaults.LogCleanerThreads, + val logCleanerIoMaxBytesPerSecond: Double = Defaults.LogCleanerIoMaxBytesPerSecond, + val logCleanerDedupeBufferSize: Long = Defaults.LogCleanerDedupeBufferSize, + val logCleanerIoBufferSize: Int = Defaults.LogCleanerIoBufferSize, + val logCleanerDedupeBufferLoadFactor: Double = Defaults.LogCleanerDedupeBufferLoadFactor, + val logCleanerBackoffMs: Long = Defaults.LogCleanerBackoffMs, + val logCleanerMinCleanRatio: Double = Defaults.LogCleanerMinCleanRatio, + val logCleanerEnable: Boolean = Defaults.LogCleanerEnable, + val logCleanerDeleteRetentionMs: Long = Defaults.LogCleanerDeleteRetentionMs, + val logIndexSizeMaxBytes: Int = Defaults.LogIndexSizeMaxBytes, + val logIndexIntervalBytes: Int = Defaults.LogIndexIntervalBytes, + val logFlushIntervalMessages: Long = Defaults.LogFlushIntervalMessages, + val logDeleteDelayMs: Long = Defaults.LogDeleteDelayMs, + val logFlushSchedulerIntervalMs: Long = Defaults.LogFlushSchedulerIntervalMs, + private val _logFlushIntervalMs: Option[Long] = None, + val logFlushOffsetCheckpointIntervalMs: Int = Defaults.LogFlushOffsetCheckpointIntervalMs, + val numRecoveryThreadsPerDataDir: Int = Defaults.NumRecoveryThreadsPerDataDir, + val autoCreateTopicsEnable: Boolean = Defaults.AutoCreateTopicsEnable, + + val minInSyncReplicas: Int = Defaults.MinInSyncReplicas, + + /** ********* Replication configuration ***********/ + val controllerSocketTimeoutMs: Int = Defaults.ControllerSocketTimeoutMs, + val controllerMessageQueueSize: Int = Defaults.ControllerMessageQueueSize, + val defaultReplicationFactor: Int = Defaults.DefaultReplicationFactor, + val replicaLagTimeMaxMs: Long = Defaults.ReplicaLagTimeMaxMs, + val replicaLagMaxMessages: Long = Defaults.ReplicaLagMaxMessages, + val replicaSocketTimeoutMs: Int = Defaults.ReplicaSocketTimeoutMs, + val replicaSocketReceiveBufferBytes: Int = Defaults.ReplicaSocketReceiveBufferBytes, + val replicaFetchMaxBytes: Int = Defaults.ReplicaFetchMaxBytes, + val replicaFetchWaitMaxMs: Int = Defaults.ReplicaFetchWaitMaxMs, + val replicaFetchMinBytes: Int = Defaults.ReplicaFetchMinBytes, + val numReplicaFetchers: Int = Defaults.NumReplicaFetchers, + val replicaHighWatermarkCheckpointIntervalMs: Long = Defaults.ReplicaHighWatermarkCheckpointIntervalMs, + val fetchPurgatoryPurgeIntervalRequests: Int = Defaults.FetchPurgatoryPurgeIntervalRequests, + val producerPurgatoryPurgeIntervalRequests: Int = Defaults.ProducerPurgatoryPurgeIntervalRequests, + val autoLeaderRebalanceEnable: Boolean = Defaults.AutoLeaderRebalanceEnable, + val leaderImbalancePerBrokerPercentage: Int = Defaults.LeaderImbalancePerBrokerPercentage, + val leaderImbalanceCheckIntervalSeconds: Int = Defaults.LeaderImbalanceCheckIntervalSeconds, + val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, + + /** ********* Controlled shutdown configuration ***********/ + val controlledShutdownMaxRetries: Int = Defaults.ControlledShutdownMaxRetries, + val controlledShutdownRetryBackoffMs: Int = Defaults.ControlledShutdownRetryBackoffMs, + val controlledShutdownEnable: Boolean = Defaults.ControlledShutdownEnable, + + /** ********* Offset management configuration ***********/ + val offsetMetadataMaxSize: Int = Defaults.OffsetMetadataMaxSize, + val offsetsLoadBufferSize: Int = Defaults.OffsetsLoadBufferSize, + val offsetsTopicReplicationFactor: Short = Defaults.OffsetsTopicReplicationFactor, + val offsetsTopicPartitions: Int = Defaults.OffsetsTopicPartitions, + val offsetsTopicSegmentBytes: Int = Defaults.OffsetsTopicSegmentBytes, + val offsetsTopicCompressionCodec: CompressionCodec = CompressionCodec.getCompressionCodec(Defaults.OffsetsTopicCompressionCodec), + val offsetsRetentionMinutes: Int = Defaults.OffsetsRetentionMinutes, + val offsetsRetentionCheckIntervalMs: Long = Defaults.OffsetsRetentionCheckIntervalMs, + val offsetCommitTimeoutMs: Int = Defaults.OffsetCommitTimeoutMs, + val offsetCommitRequiredAcks: Short = Defaults.OffsetCommitRequiredAcks, + + val deleteTopicEnable: Boolean = Defaults.DeleteTopicEnable, + val compressionType: String = Defaults.CompressionType + ) { + + val zkConnectionTimeoutMs: Int = _zkConnectionTimeoutMs.getOrElse(zkSessionTimeoutMs) + + val advertisedHostName: String = _advertisedHostName.getOrElse(hostName) + val advertisedPort: Int = _advertisedPort.getOrElse(port) + val logDirs = Utils.parseCsvList(_logDirs.getOrElse(_logDir)) + + val logRollTimeMillis = _logRollTimeMillis.getOrElse(60 * 60 * 1000L * logRollTimeHours) + val logRollTimeJitterMillis = _logRollTimeJitterMillis.getOrElse(60 * 60 * 1000L * logRollTimeJitterHours) + val logRetentionTimeMillis = getLogRetentionTimeMillis + + val logFlushIntervalMs = _logFlushIntervalMs.getOrElse(logFlushSchedulerIntervalMs) + + private def getMap(propName: String, propValue: String): Map[String, String] = { + try { + Utils.parseCsvMap(propValue) + } catch { + case e: Exception => throw new IllegalArgumentException("Error parsing configuration property '%s': %s".format(propName, e.getMessage)) + } + } + + val maxConnectionsPerIpOverrides: Map[String, Int] = + getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides).map { case (k, v) => (k, v.toInt)} + + private def getLogRetentionTimeMillis: Long = { val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute - if(props.containsKey("log.retention.ms")){ - props.getIntInRange("log.retention.ms", (1, Int.MaxValue)) - } - else if(props.containsKey("log.retention.minutes")){ - millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue)) - } - else { - millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue)) - } + _logRetentionTimeMillis.getOrElse( + _logRetentionTimeMins match { + case Some(mins) => millisInMinute * mins + case None => millisInHour * logRetentionTimeHours + } + ) } - private def getLogRollTimeMillis(): Long = { - val millisInHour = 60L * 60L * 1000L + validateValues() - if(props.containsKey("log.roll.ms")){ - props.getIntInRange("log.roll.ms", (1, Int.MaxValue)) - } - else { - millisInHour * props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue)) - } + private def validateValues() { + require(brokerId >= -1 && brokerId <= maxReservedBrokerId, "broker.id must be equal or greater than -1 and not greater than reserved.broker.max.id") + require(logRollTimeMillis >= 1, "log.roll.ms must be equal or greater than 1") + require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be equal or greater than 0") + require(logRetentionTimeMillis >= 1, "log.retention.ms must be equal or greater than 1") + require(_logRetentionTimeMins.forall(_ >= 1), "log.retention.minutes must be equal or greater than 1") + + require(logDirs.size > 0) + require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") + require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + + " to prevent unnecessary socket timeouts") + require(replicaFetchMaxBytes >= messageMaxBytes, "replica.fetch.max.bytes should be equal or greater than message.max.bytes") + require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs, "replica.fetch.wait.max.ms should always be at least replica.lag.time.max.ms" + + " to prevent frequent changes in ISR") + require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor, + "offsets.commit.required.acks must be greater or equal -1 and less or equal to offsets.topic.replication.factor") + require(BrokerCompressionCodec.isValid(compressionType), "compression.type : " + compressionType + " is not valid." + + " Valid options are " + BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) } - private def getLogRollTimeJitterMillis(): Long = { - val millisInHour = 60L * 60L * 1000L + def toProps: Properties = { + val props = new Properties() + import kafka.server.KafkaConfig._ + /** ********* Zookeeper Configuration ***********/ + props.put(ZkConnectProp, zkConnect) + props.put(ZkSessionTimeoutMsProp, zkSessionTimeoutMs.toString) + _zkConnectionTimeoutMs.foreach(value => props.put(ZkConnectionTimeoutMsProp, value.toString)) + props.put(ZkSyncTimeMsProp, zkSyncTimeMs.toString) - if(props.containsKey("log.roll.jitter.ms")) { - props.getIntInRange("log.roll.jitter.ms", (0, Int.MaxValue)) - } - else { - millisInHour * props.getIntInRange("log.roll.jitter.hours", 0, (0, Int.MaxValue)) - } + /** ********* General Configuration ***********/ + props.put(MaxReservedBrokerIdProp, maxReservedBrokerId.toString) + props.put(BrokerIdProp, brokerId.toString) + props.put(MessageMaxBytesProp, messageMaxBytes.toString) + props.put(NumNetworkThreadsProp, numNetworkThreads.toString) + props.put(NumIoThreadsProp, numIoThreads.toString) + props.put(BackgroundThreadsProp, backgroundThreads.toString) + props.put(QueuedMaxRequestsProp, queuedMaxRequests.toString) + + /** ********* Socket Server Configuration ***********/ + props.put(PortProp, port.toString) + props.put(HostNameProp, hostName) + _advertisedHostName.foreach(props.put(AdvertisedHostNameProp, _)) + _advertisedPort.foreach(value => props.put(AdvertisedPortProp, value.toString)) + props.put(SocketSendBufferBytesProp, socketSendBufferBytes.toString) + props.put(SocketReceiveBufferBytesProp, socketReceiveBufferBytes.toString) + props.put(SocketRequestMaxBytesProp, socketRequestMaxBytes.toString) + props.put(MaxConnectionsPerIpProp, maxConnectionsPerIp.toString) + props.put(MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides) + props.put(ConnectionsMaxIdleMsProp, connectionsMaxIdleMs.toString) + + /** ********* Log Configuration ***********/ + props.put(NumPartitionsProp, numPartitions.toString) + props.put(LogDirProp, _logDir) + _logDirs.foreach(value => props.put(LogDirsProp, value)) + props.put(LogSegmentBytesProp, logSegmentBytes.toString) + + props.put(LogRollTimeHoursProp, logRollTimeHours.toString) + _logRollTimeMillis.foreach(v => props.put(LogRollTimeMillisProp, v.toString)) + + props.put(LogRollTimeJitterHoursProp, logRollTimeJitterHours.toString) + _logRollTimeJitterMillis.foreach(v => props.put(LogRollTimeJitterMillisProp, v.toString)) + + + props.put(LogRetentionTimeHoursProp, logRetentionTimeHours.toString) + _logRetentionTimeMins.foreach(v => props.put(LogRetentionTimeMinutesProp, v.toString)) + _logRetentionTimeMillis.foreach(v => props.put(LogRetentionTimeMillisProp, v.toString)) + + props.put(LogRetentionBytesProp, logRetentionBytes.toString) + props.put(LogCleanupIntervalMsProp, logCleanupIntervalMs.toString) + props.put(LogCleanupPolicyProp, logCleanupPolicy) + props.put(LogCleanerThreadsProp, logCleanerThreads.toString) + props.put(LogCleanerIoMaxBytesPerSecondProp, logCleanerIoMaxBytesPerSecond.toString) + props.put(LogCleanerDedupeBufferSizeProp, logCleanerDedupeBufferSize.toString) + props.put(LogCleanerIoBufferSizeProp, logCleanerIoBufferSize.toString) + props.put(LogCleanerDedupeBufferLoadFactorProp, logCleanerDedupeBufferLoadFactor.toString) + props.put(LogCleanerBackoffMsProp, logCleanerBackoffMs.toString) + props.put(LogCleanerMinCleanRatioProp, logCleanerMinCleanRatio.toString) + props.put(LogCleanerEnableProp, logCleanerEnable.toString) + props.put(LogCleanerDeleteRetentionMsProp, logCleanerDeleteRetentionMs.toString) + props.put(LogIndexSizeMaxBytesProp, logIndexSizeMaxBytes.toString) + props.put(LogIndexIntervalBytesProp, logIndexIntervalBytes.toString) + props.put(LogFlushIntervalMessagesProp, logFlushIntervalMessages.toString) + props.put(LogDeleteDelayMsProp, logDeleteDelayMs.toString) + props.put(LogFlushSchedulerIntervalMsProp, logFlushSchedulerIntervalMs.toString) + _logFlushIntervalMs.foreach(v => props.put(LogFlushIntervalMsProp, v.toString)) + props.put(LogFlushOffsetCheckpointIntervalMsProp, logFlushOffsetCheckpointIntervalMs.toString) + props.put(NumRecoveryThreadsPerDataDirProp, numRecoveryThreadsPerDataDir.toString) + props.put(AutoCreateTopicsEnableProp, autoCreateTopicsEnable.toString) + props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) + + /** ********* Replication configuration ***********/ + props.put(ControllerSocketTimeoutMsProp, controllerSocketTimeoutMs.toString) + props.put(ControllerMessageQueueSizeProp, controllerMessageQueueSize.toString) + props.put(DefaultReplicationFactorProp, defaultReplicationFactor.toString) + props.put(ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + props.put(ReplicaLagMaxMessagesProp, replicaLagMaxMessages.toString) + props.put(ReplicaSocketTimeoutMsProp, replicaSocketTimeoutMs.toString) + props.put(ReplicaSocketReceiveBufferBytesProp, replicaSocketReceiveBufferBytes.toString) + props.put(ReplicaFetchMaxBytesProp, replicaFetchMaxBytes.toString) + props.put(ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) + props.put(ReplicaFetchMinBytesProp, replicaFetchMinBytes.toString) + props.put(NumReplicaFetchersProp, numReplicaFetchers.toString) + props.put(ReplicaHighWatermarkCheckpointIntervalMsProp, replicaHighWatermarkCheckpointIntervalMs.toString) + props.put(FetchPurgatoryPurgeIntervalRequestsProp, fetchPurgatoryPurgeIntervalRequests.toString) + props.put(ProducerPurgatoryPurgeIntervalRequestsProp, producerPurgatoryPurgeIntervalRequests.toString) + props.put(AutoLeaderRebalanceEnableProp, autoLeaderRebalanceEnable.toString) + props.put(LeaderImbalancePerBrokerPercentageProp, leaderImbalancePerBrokerPercentage.toString) + props.put(LeaderImbalanceCheckIntervalSecondsProp, leaderImbalanceCheckIntervalSeconds.toString) + props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) + + /** ********* Controlled shutdown configuration ***********/ + props.put(ControlledShutdownMaxRetriesProp, controlledShutdownMaxRetries.toString) + props.put(ControlledShutdownRetryBackoffMsProp, controlledShutdownRetryBackoffMs.toString) + props.put(ControlledShutdownEnableProp, controlledShutdownEnable.toString) + + /** ********* Offset management configuration ***********/ + props.put(OffsetMetadataMaxSizeProp, offsetMetadataMaxSize.toString) + props.put(OffsetsLoadBufferSizeProp, offsetsLoadBufferSize.toString) + props.put(OffsetsTopicReplicationFactorProp, offsetsTopicReplicationFactor.toString) + props.put(OffsetsTopicPartitionsProp, offsetsTopicPartitions.toString) + props.put(OffsetsTopicSegmentBytesProp, offsetsTopicSegmentBytes.toString) + props.put(OffsetsTopicCompressionCodecProp, offsetsTopicCompressionCodec.codec.toString) + props.put(OffsetsRetentionMinutesProp, offsetsRetentionMinutes.toString) + props.put(OffsetsRetentionCheckIntervalMsProp, offsetsRetentionCheckIntervalMs.toString) + props.put(OffsetCommitTimeoutMsProp, offsetCommitTimeoutMs.toString) + props.put(OffsetCommitRequiredAcksProp, offsetCommitRequiredAcks.toString) + props.put(DeleteTopicEnableProp, deleteTopicEnable.toString) + props.put(CompressionTypeProp, compressionType.toString) + + props } - - /*********** General Configuration ***********/ - - /* Max number that can be used for a broker.id */ - val MaxReservedBrokerId = props.getIntInRange("reserved.broker.max.id", 1000, (0, Int.MaxValue)) - - /* The broker id for this server. - * To avoid conflicts between zookeeper generated brokerId and user's config.brokerId - * added MaxReservedBrokerId and zookeeper sequence starts from MaxReservedBrokerId + 1. - */ - var brokerId: Int = if (props.containsKey("broker.id")) props.getIntInRange("broker.id", (0, MaxReservedBrokerId)) else -1 - - /* the maximum size of message that the server can receive */ - val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000 + MessageSet.LogOverhead, (0, Int.MaxValue)) - - /* the number of network threads that the server uses for handling network requests */ - val numNetworkThreads = props.getIntInRange("num.network.threads", 3, (1, Int.MaxValue)) - - /* the number of io threads that the server uses for carrying out network requests */ - val numIoThreads = props.getIntInRange("num.io.threads", 8, (1, Int.MaxValue)) - - /* the number of threads to use for various background processing tasks */ - val backgroundThreads = props.getIntInRange("background.threads", 10, (1, Int.MaxValue)) - - /* the number of queued requests allowed before blocking the network threads */ - val queuedMaxRequests = props.getIntInRange("queued.max.requests", 500, (1, Int.MaxValue)) - - /*********** Socket Server Configuration ***********/ - - /* the port to listen and accept connections on */ - val port: Int = props.getInt("port", 9092) - - /* hostname of broker. If this is set, it will only bind to this address. If this is not set, - * it will bind to all interfaces */ - val hostName: String = props.getString("host.name", null) - - /* hostname to publish to ZooKeeper for clients to use. In IaaS environments, this may - * need to be different from the interface to which the broker binds. If this is not set, - * it will use the value for "host.name" if configured. Otherwise - * it will use the value returned from java.net.InetAddress.getCanonicalHostName(). */ - val advertisedHostName: String = props.getString("advertised.host.name", hostName) - - /* the port to publish to ZooKeeper for clients to use. In IaaS environments, this may - * need to be different from the port to which the broker binds. If this is not set, - * it will publish the same port that the broker binds to. */ - val advertisedPort: Int = props.getInt("advertised.port", port) - - /* the SO_SNDBUF buffer of the socket sever sockets */ - val socketSendBufferBytes: Int = props.getInt("socket.send.buffer.bytes", 100*1024) - - /* the SO_RCVBUF buffer of the socket sever sockets */ - val socketReceiveBufferBytes: Int = props.getInt("socket.receive.buffer.bytes", 100*1024) - - /* the maximum number of bytes in a socket request */ - val socketRequestMaxBytes: Int = props.getIntInRange("socket.request.max.bytes", 100*1024*1024, (1, Int.MaxValue)) - - /* the maximum number of connections we allow from each ip address */ - val maxConnectionsPerIp: Int = props.getIntInRange("max.connections.per.ip", Int.MaxValue, (1, Int.MaxValue)) - - /* per-ip or hostname overrides to the default maximum number of connections */ - val maxConnectionsPerIpOverrides = props.getMap("max.connections.per.ip.overrides").map(entry => (entry._1, entry._2.toInt)) - - /* idle connections timeout: the server socket processor threads close the connections that idle more than this */ - val connectionsMaxIdleMs = props.getLong("connections.max.idle.ms", 10*60*1000L) - - /*********** Log Configuration ***********/ - - /* the default number of log partitions per topic */ - val numPartitions = props.getIntInRange("num.partitions", 1, (1, Int.MaxValue)) - - /* the directories in which the log data is kept */ - val logDirs = Utils.parseCsvList(props.getString("log.dirs", props.getString("log.dir", "/tmp/kafka-logs"))) - require(logDirs.size > 0) - - /* the maximum size of a single log file */ - val logSegmentBytes = props.getIntInRange("log.segment.bytes", 1*1024*1024*1024, (Message.MinHeaderSize, Int.MaxValue)) - - /* the maximum time before a new log segment is rolled out */ - val logRollTimeMillis = getLogRollTimeMillis - - /* the maximum jitter to subtract from logRollTimeMillis */ - val logRollTimeJitterMillis = getLogRollTimeJitterMillis - - /* the number of hours to keep a log file before deleting it */ - val logRetentionTimeMillis = getLogRetentionTimeMillis - - /* the maximum size of the log before deleting it */ - val logRetentionBytes = props.getLong("log.retention.bytes", -1) - - /* the frequency in minutes that the log cleaner checks whether any log is eligible for deletion */ - val logCleanupIntervalMs = props.getLongInRange("log.retention.check.interval.ms", 5*60*1000, (1, Long.MaxValue)) - - /* the default cleanup policy for segments beyond the retention window, must be either "delete" or "compact" */ - val logCleanupPolicy = props.getString("log.cleanup.policy", "delete") - - /* the number of background threads to use for log cleaning */ - val logCleanerThreads = props.getIntInRange("log.cleaner.threads", 1, (0, Int.MaxValue)) - - /* the log cleaner will be throttled so that the sum of its read and write i/o will be less than this value on average */ - val logCleanerIoMaxBytesPerSecond = props.getDouble("log.cleaner.io.max.bytes.per.second", Double.MaxValue) - - /* the total memory used for log deduplication across all cleaner threads */ - val logCleanerDedupeBufferSize = props.getLongInRange("log.cleaner.dedupe.buffer.size", 500*1024*1024L, (0, Long.MaxValue)) - require(logCleanerDedupeBufferSize / logCleanerThreads > 1024*1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") - - /* the total memory used for log cleaner I/O buffers across all cleaner threads */ - val logCleanerIoBufferSize = props.getIntInRange("log.cleaner.io.buffer.size", 512*1024, (0, Int.MaxValue)) - - /* log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value - * will allow more log to be cleaned at once but will lead to more hash collisions */ - val logCleanerDedupeBufferLoadFactor = props.getDouble("log.cleaner.io.buffer.load.factor", 0.9d) - - /* the amount of time to sleep when there are no logs to clean */ - val logCleanerBackoffMs = props.getLongInRange("log.cleaner.backoff.ms", 15*1000, (0L, Long.MaxValue)) - - /* the minimum ratio of dirty log to total log for a log to eligible for cleaning */ - val logCleanerMinCleanRatio = props.getDouble("log.cleaner.min.cleanable.ratio", 0.5) - - /* should we enable log cleaning? */ - val logCleanerEnable = props.getBoolean("log.cleaner.enable", false) - - /* how long are delete records retained? */ - val logCleanerDeleteRetentionMs = props.getLong("log.cleaner.delete.retention.ms", 24 * 60 * 60 * 1000L) - - /* the maximum size in bytes of the offset index */ - val logIndexSizeMaxBytes = props.getIntInRange("log.index.size.max.bytes", 10*1024*1024, (4, Int.MaxValue)) - - /* the interval with which we add an entry to the offset index */ - val logIndexIntervalBytes = props.getIntInRange("log.index.interval.bytes", 4096, (0, Int.MaxValue)) - - /* the number of messages accumulated on a log partition before messages are flushed to disk */ - val logFlushIntervalMessages = props.getLongInRange("log.flush.interval.messages", Long.MaxValue, (1, Long.MaxValue)) - - /* the amount of time to wait before deleting a file from the filesystem */ - val logDeleteDelayMs = props.getLongInRange("log.segment.delete.delay.ms", 60000, (0, Long.MaxValue)) - - /* the frequency in ms that the log flusher checks whether any log needs to be flushed to disk */ - val logFlushSchedulerIntervalMs = props.getLong("log.flush.scheduler.interval.ms", Long.MaxValue) - - /* the maximum time in ms that a message in any topic is kept in memory before flushed to disk */ - val logFlushIntervalMs = props.getLong("log.flush.interval.ms", logFlushSchedulerIntervalMs) - - /* the frequency with which we update the persistent record of the last flush which acts as the log recovery point */ - val logFlushOffsetCheckpointIntervalMs = props.getIntInRange("log.flush.offset.checkpoint.interval.ms", 60000, (0, Int.MaxValue)) - - /* the number of threads per data directory to be used for log recovery at startup and flushing at shutdown */ - val numRecoveryThreadsPerDataDir = props.getIntInRange("num.recovery.threads.per.data.dir", 1, (1, Int.MaxValue)) - - /* enable auto creation of topic on the server */ - val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true) - - /* define the minimum number of replicas in ISR needed to satisfy a produce request with required.acks=-1 (or all) */ - val minInSyncReplicas = props.getIntInRange("min.insync.replicas",1,(1,Int.MaxValue)) - - - - /*********** Replication configuration ***********/ - - /* the socket timeout for controller-to-broker channels */ - val controllerSocketTimeoutMs = props.getInt("controller.socket.timeout.ms", 30000) - - /* the buffer size for controller-to-broker-channels */ - val controllerMessageQueueSize= props.getInt("controller.message.queue.size", Int.MaxValue) - - /* default replication factors for automatically created topics */ - val defaultReplicationFactor = props.getInt("default.replication.factor", 1) - - /* If a follower hasn't sent any fetch requests during this time, the leader will remove the follower from isr */ - val replicaLagTimeMaxMs = props.getLong("replica.lag.time.max.ms", 10000) - - /* If the lag in messages between a leader and a follower exceeds this number, the leader will remove the follower from isr */ - val replicaLagMaxMessages = props.getLong("replica.lag.max.messages", 4000) - - /* the socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms. */ - val replicaSocketTimeoutMs = props.getInt("replica.socket.timeout.ms", ConsumerConfig.SocketTimeout) - require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + - " to prevent unnecessary socket timeouts") - - /* the socket receive buffer for network requests */ - val replicaSocketReceiveBufferBytes = props.getInt("replica.socket.receive.buffer.bytes", ConsumerConfig.SocketBufferSize) - - /* the number of byes of messages to attempt to fetch */ - val replicaFetchMaxBytes = props.getIntInRange("replica.fetch.max.bytes", ConsumerConfig.FetchSize, (messageMaxBytes, Int.MaxValue)) - - /* max wait time for each fetcher request issued by follower replicas. This value should always be less than the - * replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics */ - val replicaFetchWaitMaxMs = props.getInt("replica.fetch.wait.max.ms", 500) - require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs, "replica.fetch.wait.max.ms should always be at least replica.lag.time.max.ms" + - " to prevent frequent changes in ISR") - - /* minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs */ - val replicaFetchMinBytes = props.getInt("replica.fetch.min.bytes", 1) - - /* number of fetcher threads used to replicate messages from a source broker. - * Increasing this value can increase the degree of I/O parallelism in the follower broker. */ - val numReplicaFetchers = props.getInt("num.replica.fetchers", 1) - - /* the frequency with which the high watermark is saved out to disk */ - val replicaHighWatermarkCheckpointIntervalMs = props.getLong("replica.high.watermark.checkpoint.interval.ms", 5000L) - - /* the purge interval (in number of requests) of the fetch request purgatory */ - val fetchPurgatoryPurgeIntervalRequests = props.getInt("fetch.purgatory.purge.interval.requests", 1000) - - /* the purge interval (in number of requests) of the producer request purgatory */ - val producerPurgatoryPurgeIntervalRequests = props.getInt("producer.purgatory.purge.interval.requests", 1000) - - /* Enables auto leader balancing. A background thread checks and triggers leader - * balance if required at regular intervals */ - val autoLeaderRebalanceEnable = props.getBoolean("auto.leader.rebalance.enable", true) - - /* the ratio of leader imbalance allowed per broker. The controller would trigger a leader balance if it goes above - * this value per broker. The value is specified in percentage. */ - val leaderImbalancePerBrokerPercentage = props.getInt("leader.imbalance.per.broker.percentage", 10) - - /* the frequency with which the partition rebalance check is triggered by the controller */ - val leaderImbalanceCheckIntervalSeconds = props.getInt("leader.imbalance.check.interval.seconds", 300) - - /* indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though - * doing so may result in data loss */ - val uncleanLeaderElectionEnable = props.getBoolean("unclean.leader.election.enable", true) - - /*********** Controlled shutdown configuration ***********/ - - /** Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens */ - val controlledShutdownMaxRetries = props.getInt("controlled.shutdown.max.retries", 3) - - /** Before each retry, the system needs time to recover from the state that caused the previous failure (Controller - * fail over, replica lag etc). This config determines the amount of time to wait before retrying. */ - val controlledShutdownRetryBackoffMs = props.getInt("controlled.shutdown.retry.backoff.ms", 5000) - - /* enable controlled shutdown of the server */ - val controlledShutdownEnable = props.getBoolean("controlled.shutdown.enable", default = true) - - /*********** Offset management configuration ***********/ - - /* the maximum size for a metadata entry associated with an offset commit */ - val offsetMetadataMaxSize = props.getInt("offset.metadata.max.bytes", OffsetManagerConfig.DefaultMaxMetadataSize) - - /** Batch size for reading from the offsets segments when loading offsets into the cache. */ - val offsetsLoadBufferSize = props.getIntInRange("offsets.load.buffer.size", - OffsetManagerConfig.DefaultLoadBufferSize, (1, Integer.MAX_VALUE)) - - /** The replication factor for the offsets topic (set higher to ensure availability). To - * ensure that the effective replication factor of the offsets topic is the configured value, - * the number of alive brokers has to be at least the replication factor at the time of the - * first request for the offsets topic. If not, either the offsets topic creation will fail or - * it will get a replication factor of min(alive brokers, configured replication factor) */ - val offsetsTopicReplicationFactor: Short = props.getShortInRange("offsets.topic.replication.factor", - OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor, (1, Short.MaxValue)) - - /** The number of partitions for the offset commit topic (should not change after deployment). */ - val offsetsTopicPartitions: Int = props.getIntInRange("offsets.topic.num.partitions", - OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, (1, Integer.MAX_VALUE)) - - /** The offsets topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads */ - val offsetsTopicSegmentBytes: Int = props.getIntInRange("offsets.topic.segment.bytes", - OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, (1, Integer.MAX_VALUE)) - - /** Compression codec for the offsets topic - compression may be used to achieve "atomic" commits. */ - val offsetsTopicCompressionCodec = props.getCompressionCodec("offsets.topic.compression.codec", - OffsetManagerConfig.DefaultOffsetsTopicCompressionCodec) - - /** Offsets older than this retention period will be discarded. */ - val offsetsRetentionMinutes: Int = props.getIntInRange("offsets.retention.minutes", 24*60, (1, Integer.MAX_VALUE)) - - /** Frequency at which to check for stale offsets. */ - val offsetsRetentionCheckIntervalMs: Long = props.getLongInRange("offsets.retention.check.interval.ms", - OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, (1, Long.MaxValue)) - - /* Offset commit will be delayed until all replicas for the offsets topic receive the commit or this timeout is - * reached. This is similar to the producer request timeout. */ - val offsetCommitTimeoutMs = props.getIntInRange("offsets.commit.timeout.ms", - OffsetManagerConfig.DefaultOffsetCommitTimeoutMs, (1, Integer.MAX_VALUE)) - - /** The required acks before the commit can be accepted. In general, the default (-1) should not be overridden. */ - val offsetCommitRequiredAcks = props.getShortInRange("offsets.commit.required.acks", - OffsetManagerConfig.DefaultOffsetCommitRequiredAcks, (-1, offsetsTopicReplicationFactor)) - - /* Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off */ - val deleteTopicEnable = props.getBoolean("delete.topic.enable", false) - - /** - * Specify the final compression type for a given topic. This configuration accepts the standard compression codecs - * ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to no compression; and - * 'producer' which means retain the original compression codec set by the producer." - */ - val compressionType = props.getString("compression.type", "producer").toLowerCase() - require(BrokerCompressionCodec.isValid(compressionType), "compression.type : "+compressionType + " is not valid." + - " Valid options are "+BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) -} +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 8e3def9e9ed..378a74d9e8e 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -453,7 +453,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg private def generateBrokerId: Int = { try { - ZkUtils.getBrokerSequenceId(zkClient, config.MaxReservedBrokerId) + ZkUtils.getBrokerSequenceId(zkClient, config.maxReservedBrokerId) } catch { case e: Exception => error("Failed to generate broker.id due to ", e) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 6879e730282..d6d14fbd167 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -89,7 +89,7 @@ class ReplicaFetcherThread(name:String, // Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election. // This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise, // we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration. - if (!LogConfig.fromProps(brokerConfig.props.props, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, + if (!LogConfig.fromProps(brokerConfig.toProps, AdminUtils.fetchTopicConfig(replicaMgr.zkClient, topicAndPartition.topic)).uncleanLeaderElectionEnable) { // Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur. fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) + diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 5650b4a7b95..82fe4c9a138 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -44,7 +44,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { override lazy val configs = { val cfgs = TestUtils.createBrokerConfigs(serverCount) cfgs.map(_.putAll(serverConfig)) - cfgs.map(new KafkaConfig(_)) + cfgs.map(KafkaConfig.fromProps) } var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index e6355888927..cae72f4f87f 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -44,7 +44,7 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK private var server: KafkaServer = null private val props = TestUtils.createBrokerConfig(brokerId, port) - private val config = new KafkaConfig(props) + private val config = KafkaConfig.fromProps(props) private val topic = "topic" private val numRecords = 2000 diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index d34ee3a40dc..8246e128109 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -40,17 +40,18 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { private val serverMessageMaxBytes = producerBufferSize/2 val numServers = 2 - val configs = - for(props <- TestUtils.createBrokerConfigs(numServers, false)) - yield new KafkaConfig(props) { - override val zkConnect = TestZKUtils.zookeeperConnect - override val autoCreateTopicsEnable = false - override val messageMaxBytes = serverMessageMaxBytes - // 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 - override val offsetsTopicPartitions = 1 - } + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ZkConnectProp, TestZKUtils.zookeeperConnect) + overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, 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 + overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, 1.toString) + + val configs = + for (props <- TestUtils.createBrokerConfigs(numServers, false)) + yield KafkaConfig.fromProps(props, overridingProps) private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 8154a4210dc..3df45078459 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -37,12 +37,14 @@ import org.apache.kafka.common.serialization.ByteArraySerializer class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { val numServers = 2 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ZkConnectProp, TestZKUtils.zookeeperConnect) + overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) + val configs = - for(props <- TestUtils.createBrokerConfigs(numServers, false)) - yield new KafkaConfig(props) { - override val zkConnect = TestZKUtils.zookeeperConnect - override val numPartitions = 4 - } + for (props <- TestUtils.createBrokerConfigs(numServers, false)) + yield KafkaConfig.fromProps(props, overridingProps) private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 1bf2667f478..8bc178505e0 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -55,10 +55,10 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() // start all the servers - val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) - val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) - val server3 = TestUtils.createServer(new KafkaConfig(configProps3)) - val server4 = TestUtils.createServer(new KafkaConfig(configProps4)) + val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) + val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) + val server3 = TestUtils.createServer(KafkaConfig.fromProps(configProps3)) + val server4 = TestUtils.createServer(KafkaConfig.fromProps(configProps4)) servers ++= List(server1, server2, server3, server4) brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.config.port)) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index e2897982711..ee0b21e6a94 100644 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -145,7 +145,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -176,7 +176,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -207,7 +207,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val expectedReplicaAssignment = Map(0 -> List(0, 1)) val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // reassign partition 0 @@ -236,7 +236,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { def testReassigningNonExistingPartition() { val topic = "test" // create brokers - val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(4, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // reassign partition 0 val newReplicas = Seq(2, 3) val partitionToBeReassigned = 0 @@ -262,7 +262,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas)) reassignPartitionsCommand.reassignPartitions // create brokers - val servers = TestUtils.createBrokerConfigs(2, false).map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = TestUtils.createBrokerConfigs(2, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // wait until reassignment completes TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkClient), @@ -298,7 +298,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val partition = 1 val preferredReplica = 0 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(new KafkaConfig(_)) + val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(KafkaConfig.fromProps) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) @@ -318,7 +318,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { val topic = "test" val partition = 1 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(new KafkaConfig(_)) + val serverConfigs = TestUtils.createBrokerConfigs(3, false).map(KafkaConfig.fromProps) val servers = serverConfigs.reverse.map(s => TestUtils.createServer(s)) // create the topic TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers) @@ -365,7 +365,7 @@ class AdminTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { def testTopicConfigChange() { val partitions = 3 val topic = "my-topic" - val server = TestUtils.createServer(new KafkaConfig(TestUtils.createBrokerConfig(0))) + val server = TestUtils.createServer(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0))) def makeConfig(messageSize: Int, retentionMs: Long) = { var props = new Properties() diff --git a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala index d530338728b..1baff0ea982 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala @@ -26,7 +26,7 @@ import kafka.integration.KafkaServerTestHarness class DeleteConsumerGroupTest extends JUnit3Suite with KafkaServerTestHarness { - val configs = TestUtils.createBrokerConfigs(3, false, true).map(new KafkaConfig(_)) + val configs = TestUtils.createBrokerConfigs(3, false, true).map(KafkaConfig.fromProps) @Test def testGroupWideDeleteInZK() { diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index c8f336aa034..6258983451b 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -99,7 +99,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val brokerConfigs = TestUtils.createBrokerConfigs(4, false) brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) // create brokers - val allServers = brokerConfigs.map(b => TestUtils.createServer(new KafkaConfig(b))) + val allServers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId)) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) @@ -263,7 +263,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topicAndPartition = TopicAndPartition(topic, 0) // create brokers - val servers = brokerConfigs.map(b => TestUtils.createServer(new KafkaConfig(b))) + val servers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b))) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until replica log is created on every broker diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index c0355cc0135..995397ba2e2 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -18,6 +18,7 @@ package kafka.consumer +import java.util.Properties import java.util.concurrent._ import java.util.concurrent.atomic._ import scala.collection._ @@ -36,11 +37,14 @@ import kafka.integration.KafkaServerTestHarness class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val numNodes = 1 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ZkConnectProp, TestZKUtils.zookeeperConnect) + val configs = for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val zkConnect = TestZKUtils.zookeeperConnect - } + yield KafkaConfig.fromProps(props, overridingProps) + val messages = new mutable.HashMap[Int, Seq[Message]] val topic = "topic" val group = "group1" diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index a17e8532c44..19640cc55b5 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -42,12 +42,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val numNodes = 2 val numParts = 2 val topic = "topic1" + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ZkConnectProp, zookeeperConnect) + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val zkConnect = zookeeperConnect - override val numPartitions = numParts - } + for (props <- TestUtils.createBrokerConfigs(numNodes)) + yield KafkaConfig.fromProps(props, overridingProps) + val group = "group1" val consumer0 = "consumer0" val consumer1 = "consumer1" diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala index 95303e098d4..ffa6c306a44 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -31,7 +31,7 @@ import junit.framework.Assert._ class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging { - val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0))) + val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0))) val topic = "test_topic" val group = "default_group" diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 25845abbcad..3093e459935 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -36,7 +36,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { val numNodes = 1 val configs = for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) + yield KafkaConfig.fromProps(props) val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] val topic = "topic" val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index aeb7a19acae..30deaf47b64 100644 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -42,7 +42,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val port = TestUtils.choosePort() val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) val configs = List(config) def testFetchRequestCanProperlySerialize() { diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala index eab4b5f6190..4d27e41c727 100644 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -49,10 +49,10 @@ class RollingBounceTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() // start all the servers - val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) - val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) - val server3 = TestUtils.createServer(new KafkaConfig(configProps3)) - val server4 = TestUtils.createServer(new KafkaConfig(configProps4)) + val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) + val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) + val server3 = TestUtils.createServer(KafkaConfig.fromProps(configProps3)) + val server4 = TestUtils.createServer(KafkaConfig.fromProps(configProps4)) servers ++= List(server1, server2, server3, server4) } diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index 35dc071b105..a671af4a87d 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -32,7 +32,7 @@ import kafka.client.ClientUtils class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(1) - val configs = props.map(p => new KafkaConfig(p)) + val configs = props.map(p => KafkaConfig.fromProps(p)) private var server1: KafkaServer = null val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port)) diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index ba3bcdcd1de..8342cae564e 100644 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -91,7 +91,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { private def startBrokers(cluster: Seq[Properties]) { for (props <- cluster) { - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) val server = createServer(config) configs ++= List(config) servers ++= List(server) diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala index d6248b09bb0..3d0fc9deda2 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -17,6 +17,8 @@ package kafka.javaapi.consumer +import java.util.Properties + import kafka.server._ import kafka.message._ import kafka.serializer._ @@ -42,12 +44,15 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val numNodes = 2 val numParts = 2 val topic = "topic1" + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ZkConnectProp, zookeeperConnect) + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + val configs = - for(props <- TestUtils.createBrokerConfigs(numNodes)) - yield new KafkaConfig(props) { - override val numPartitions = numParts - override val zkConnect = zookeeperConnect - } + for (props <- TestUtils.createBrokerConfigs(numNodes)) + yield KafkaConfig.fromProps(props, overridingProps) + val group = "group1" val consumer1 = "consumer1" val nMessages = 2 diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 1a4be70a21f..8cd5f2fa4a1 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -38,7 +38,7 @@ class LogTest extends JUnitSuite { def setUp() { logDir = TestUtils.tempDir() val props = TestUtils.createBrokerConfig(0, -1) - config = new KafkaConfig(props) + config = KafkaConfig.fromProps(props) } @After diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala index 4ea0489c9fd..36db9172ea2 100644 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala @@ -57,7 +57,7 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with val propsZk = TestUtils.createBrokerConfig(brokerZk, portZk) val logDirZkPath = propsZk.getProperty("log.dir") logDirZk = new File(logDirZkPath) - config = new KafkaConfig(propsZk) + config = KafkaConfig.fromProps(propsZk) server = TestUtils.createServer(config) simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64 * 1024, "") } diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index 111e4a26c1e..0f58ad8e698 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -17,6 +17,8 @@ package kafka.consumer +import java.util.Properties + import com.yammer.metrics.Metrics import com.yammer.metrics.core.MetricPredicate import org.junit.Test @@ -38,12 +40,15 @@ class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { val numNodes = 2 val numParts = 2 val topic = "topic1" + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ZkConnectProp, zookeeperConnect) + overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) + val configs = - for (props <- TestUtils.createBrokerConfigs(numNodes, enableDeleteTopic=true)) - yield new KafkaConfig(props) { - override val zkConnect = zookeeperConnect - override val numPartitions = numParts - } + for (props <- TestUtils.createBrokerConfigs(numNodes, enableDeleteTopic = true)) + yield KafkaConfig.fromProps(props, overridingProps) + val nMessages = 2 override def tearDown() { diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index 1db6ac329f7..be90c5bc7f1 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -37,7 +37,7 @@ import kafka.utils._ class AsyncProducerTest extends JUnit3Suite { val props = createBrokerConfigs(1) - val configs = props.map(p => new KafkaConfig(p)) + val configs = props.map(p => KafkaConfig.fromProps(p)) override def setUp() { super.setUp() diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index ce65dab4910..d2f3851cb68 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -51,10 +51,10 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false) props1.put("num.partitions", "4") - private val config1 = new KafkaConfig(props1) + private val config1 = KafkaConfig.fromProps(props1) private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false) props2.put("num.partitions", "4") - private val config2 = new KafkaConfig(props2) + private val config2 = KafkaConfig.fromProps(props2) override def setUp() { super.setUp() diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index d60d8e0f494..b5208a5f118 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -33,7 +33,7 @@ import kafka.common.{TopicAndPartition, ErrorMapping} class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { private var messageBytes = new Array[Byte](2); // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. - val configs = List(new KafkaConfig(TestUtils.createBrokerConfigs(1, false).head)) + val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, false).head)) val zookeeperConnect = TestZKUtils.zookeeperConnect @Test diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index f0c4a56b61b..296e2b56304 100644 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -34,7 +34,7 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { props.put("advertised.host.name", advertisedHostName) props.put("advertised.port", advertisedPort.toString) - server = TestUtils.createServer(new KafkaConfig(props)) + server = TestUtils.createServer(KafkaConfig.fromProps(props)) } override def tearDown() { diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index ad121169a5e..93182aeb342 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -27,7 +27,7 @@ import org.scalatest.junit.JUnit3Suite class DynamicConfigChangeTest extends JUnit3Suite with KafkaServerTestHarness { - override val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0, TestUtils.choosePort))) + override val configs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, TestUtils.choosePort))) @Test def testConfigChange() { diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 8913fc1d59f..0bdbc2f31fc 100644 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean class HighwatermarkPersistenceTest extends JUnit3Suite { - val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_)) + val configs = TestUtils.createBrokerConfigs(2).map(KafkaConfig.fromProps) val topic = "foo" val logManagers = configs map { config => TestUtils.createLogManager( diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index a703d271504..92152358c95 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -16,6 +16,8 @@ */ package kafka.server +import java.util.Properties + import org.scalatest.junit.JUnit3Suite import collection.mutable.HashMap import collection.mutable.Map @@ -29,11 +31,15 @@ import java.util.concurrent.atomic.AtomicBoolean class IsrExpirationTest extends JUnit3Suite { var topicPartitionIsr: Map[(String, Int), Seq[Int]] = new HashMap[(String, Int), Seq[Int]]() - val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) { - override val replicaLagTimeMaxMs = 100L - override val replicaFetchWaitMaxMs = 100 - override val replicaLagMaxMessages = 10L - }) + val replicaLagTimeMaxMs = 100L + val replicaFetchWaitMaxMs = 100 + val replicaLagMaxMessages = 10L + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaLagMaxMessagesProp, replicaLagMaxMessages.toString) + val configs = TestUtils.createBrokerConfigs(2).map(KafkaConfig.fromProps(_, overridingProps)) val topic = "foo" val time = new MockTime diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala new file mode 100644 index 00000000000..c124c8df5b5 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -0,0 +1,403 @@ +/** + * 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 unit.kafka.server + +import java.util.Properties + +import kafka.message._ +import kafka.server.{Defaults, KafkaConfig} +import org.apache.kafka.common.config.ConfigException +import org.junit.{Assert, Test} +import org.scalatest.junit.JUnit3Suite + +import scala.collection.Map +import scala.util.Random._ + +class KafkaConfigConfigDefTest extends JUnit3Suite { + + @Test + def testFromPropsDefaults() { + val defaults = new Properties() + defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + + // some ordinary setting + defaults.put(KafkaConfig.AdvertisedPortProp, "1818") + + val props = new Properties(defaults) + + val config = KafkaConfig.fromProps(props) + + Assert.assertEquals(1818, config.advertisedPort) + Assert.assertEquals("KafkaConfig defaults should be retained", Defaults.ConnectionsMaxIdleMs, config.connectionsMaxIdleMs) + } + + @Test + def testFromPropsEmpty() { + // only required + val p = new Properties() + p.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + val actualConfig = KafkaConfig.fromProps(p) + + val expectedConfig = new KafkaConfig(zkConnect = "127.0.0.1:2181") + + Assert.assertEquals(expectedConfig.zkConnect, actualConfig.zkConnect) + Assert.assertEquals(expectedConfig.zkSessionTimeoutMs, actualConfig.zkSessionTimeoutMs) + Assert.assertEquals(expectedConfig.zkConnectionTimeoutMs, actualConfig.zkConnectionTimeoutMs) + Assert.assertEquals(expectedConfig.zkSyncTimeMs, actualConfig.zkSyncTimeMs) + Assert.assertEquals(expectedConfig.maxReservedBrokerId, actualConfig.maxReservedBrokerId) + Assert.assertEquals(expectedConfig.brokerId, actualConfig.brokerId) + Assert.assertEquals(expectedConfig.messageMaxBytes, actualConfig.messageMaxBytes) + Assert.assertEquals(expectedConfig.numNetworkThreads, actualConfig.numNetworkThreads) + Assert.assertEquals(expectedConfig.numIoThreads, actualConfig.numIoThreads) + Assert.assertEquals(expectedConfig.backgroundThreads, actualConfig.backgroundThreads) + Assert.assertEquals(expectedConfig.queuedMaxRequests, actualConfig.queuedMaxRequests) + + Assert.assertEquals(expectedConfig.port, actualConfig.port) + Assert.assertEquals(expectedConfig.hostName, actualConfig.hostName) + Assert.assertEquals(expectedConfig.advertisedHostName, actualConfig.advertisedHostName) + Assert.assertEquals(expectedConfig.advertisedPort, actualConfig.advertisedPort) + Assert.assertEquals(expectedConfig.socketSendBufferBytes, actualConfig.socketSendBufferBytes) + Assert.assertEquals(expectedConfig.socketReceiveBufferBytes, actualConfig.socketReceiveBufferBytes) + Assert.assertEquals(expectedConfig.socketRequestMaxBytes, actualConfig.socketRequestMaxBytes) + Assert.assertEquals(expectedConfig.maxConnectionsPerIp, actualConfig.maxConnectionsPerIp) + Assert.assertEquals(expectedConfig.maxConnectionsPerIpOverrides, actualConfig.maxConnectionsPerIpOverrides) + Assert.assertEquals(expectedConfig.connectionsMaxIdleMs, actualConfig.connectionsMaxIdleMs) + + Assert.assertEquals(expectedConfig.numPartitions, actualConfig.numPartitions) + Assert.assertEquals(expectedConfig.logDirs, actualConfig.logDirs) + + Assert.assertEquals(expectedConfig.logSegmentBytes, actualConfig.logSegmentBytes) + + Assert.assertEquals(expectedConfig.logRollTimeMillis, actualConfig.logRollTimeMillis) + Assert.assertEquals(expectedConfig.logRollTimeJitterMillis, actualConfig.logRollTimeJitterMillis) + Assert.assertEquals(expectedConfig.logRetentionTimeMillis, actualConfig.logRetentionTimeMillis) + + Assert.assertEquals(expectedConfig.logRetentionBytes, actualConfig.logRetentionBytes) + Assert.assertEquals(expectedConfig.logCleanupIntervalMs, actualConfig.logCleanupIntervalMs) + Assert.assertEquals(expectedConfig.logCleanupPolicy, actualConfig.logCleanupPolicy) + Assert.assertEquals(expectedConfig.logCleanerThreads, actualConfig.logCleanerThreads) + Assert.assertEquals(expectedConfig.logCleanerIoMaxBytesPerSecond, actualConfig.logCleanerIoMaxBytesPerSecond, 0.0) + Assert.assertEquals(expectedConfig.logCleanerDedupeBufferSize, actualConfig.logCleanerDedupeBufferSize) + Assert.assertEquals(expectedConfig.logCleanerIoBufferSize, actualConfig.logCleanerIoBufferSize) + Assert.assertEquals(expectedConfig.logCleanerDedupeBufferLoadFactor, actualConfig.logCleanerDedupeBufferLoadFactor, 0.0) + Assert.assertEquals(expectedConfig.logCleanerBackoffMs, actualConfig.logCleanerBackoffMs) + Assert.assertEquals(expectedConfig.logCleanerMinCleanRatio, actualConfig.logCleanerMinCleanRatio, 0.0) + Assert.assertEquals(expectedConfig.logCleanerEnable, actualConfig.logCleanerEnable) + Assert.assertEquals(expectedConfig.logCleanerDeleteRetentionMs, actualConfig.logCleanerDeleteRetentionMs) + Assert.assertEquals(expectedConfig.logIndexSizeMaxBytes, actualConfig.logIndexSizeMaxBytes) + Assert.assertEquals(expectedConfig.logIndexIntervalBytes, actualConfig.logIndexIntervalBytes) + Assert.assertEquals(expectedConfig.logFlushIntervalMessages, actualConfig.logFlushIntervalMessages) + Assert.assertEquals(expectedConfig.logDeleteDelayMs, actualConfig.logDeleteDelayMs) + Assert.assertEquals(expectedConfig.logFlushSchedulerIntervalMs, actualConfig.logFlushSchedulerIntervalMs) + Assert.assertEquals(expectedConfig.logFlushIntervalMs, actualConfig.logFlushIntervalMs) + Assert.assertEquals(expectedConfig.logFlushOffsetCheckpointIntervalMs, actualConfig.logFlushOffsetCheckpointIntervalMs) + Assert.assertEquals(expectedConfig.numRecoveryThreadsPerDataDir, actualConfig.numRecoveryThreadsPerDataDir) + Assert.assertEquals(expectedConfig.autoCreateTopicsEnable, actualConfig.autoCreateTopicsEnable) + + Assert.assertEquals(expectedConfig.minInSyncReplicas, actualConfig.minInSyncReplicas) + + Assert.assertEquals(expectedConfig.controllerSocketTimeoutMs, actualConfig.controllerSocketTimeoutMs) + Assert.assertEquals(expectedConfig.controllerMessageQueueSize, actualConfig.controllerMessageQueueSize) + Assert.assertEquals(expectedConfig.defaultReplicationFactor, actualConfig.defaultReplicationFactor) + Assert.assertEquals(expectedConfig.replicaLagTimeMaxMs, actualConfig.replicaLagTimeMaxMs) + Assert.assertEquals(expectedConfig.replicaLagMaxMessages, actualConfig.replicaLagMaxMessages) + Assert.assertEquals(expectedConfig.replicaSocketTimeoutMs, actualConfig.replicaSocketTimeoutMs) + Assert.assertEquals(expectedConfig.replicaSocketReceiveBufferBytes, actualConfig.replicaSocketReceiveBufferBytes) + Assert.assertEquals(expectedConfig.replicaFetchMaxBytes, actualConfig.replicaFetchMaxBytes) + Assert.assertEquals(expectedConfig.replicaFetchWaitMaxMs, actualConfig.replicaFetchWaitMaxMs) + Assert.assertEquals(expectedConfig.replicaFetchMinBytes, actualConfig.replicaFetchMinBytes) + Assert.assertEquals(expectedConfig.numReplicaFetchers, actualConfig.numReplicaFetchers) + Assert.assertEquals(expectedConfig.replicaHighWatermarkCheckpointIntervalMs, actualConfig.replicaHighWatermarkCheckpointIntervalMs) + Assert.assertEquals(expectedConfig.fetchPurgatoryPurgeIntervalRequests, actualConfig.fetchPurgatoryPurgeIntervalRequests) + Assert.assertEquals(expectedConfig.producerPurgatoryPurgeIntervalRequests, actualConfig.producerPurgatoryPurgeIntervalRequests) + Assert.assertEquals(expectedConfig.autoLeaderRebalanceEnable, actualConfig.autoLeaderRebalanceEnable) + Assert.assertEquals(expectedConfig.leaderImbalancePerBrokerPercentage, actualConfig.leaderImbalancePerBrokerPercentage) + Assert.assertEquals(expectedConfig.leaderImbalanceCheckIntervalSeconds, actualConfig.leaderImbalanceCheckIntervalSeconds) + Assert.assertEquals(expectedConfig.uncleanLeaderElectionEnable, actualConfig.uncleanLeaderElectionEnable) + + Assert.assertEquals(expectedConfig.controlledShutdownMaxRetries, actualConfig.controlledShutdownMaxRetries) + Assert.assertEquals(expectedConfig.controlledShutdownRetryBackoffMs, actualConfig.controlledShutdownRetryBackoffMs) + Assert.assertEquals(expectedConfig.controlledShutdownEnable, actualConfig.controlledShutdownEnable) + + Assert.assertEquals(expectedConfig.offsetMetadataMaxSize, actualConfig.offsetMetadataMaxSize) + Assert.assertEquals(expectedConfig.offsetsLoadBufferSize, actualConfig.offsetsLoadBufferSize) + Assert.assertEquals(expectedConfig.offsetsTopicReplicationFactor, actualConfig.offsetsTopicReplicationFactor) + Assert.assertEquals(expectedConfig.offsetsTopicPartitions, actualConfig.offsetsTopicPartitions) + Assert.assertEquals(expectedConfig.offsetsTopicSegmentBytes, actualConfig.offsetsTopicSegmentBytes) + Assert.assertEquals(expectedConfig.offsetsTopicCompressionCodec, actualConfig.offsetsTopicCompressionCodec) + Assert.assertEquals(expectedConfig.offsetsRetentionMinutes, actualConfig.offsetsRetentionMinutes) + Assert.assertEquals(expectedConfig.offsetsRetentionCheckIntervalMs, actualConfig.offsetsRetentionCheckIntervalMs) + Assert.assertEquals(expectedConfig.offsetCommitTimeoutMs, actualConfig.offsetCommitTimeoutMs) + Assert.assertEquals(expectedConfig.offsetCommitRequiredAcks, actualConfig.offsetCommitRequiredAcks) + + Assert.assertEquals(expectedConfig.deleteTopicEnable, actualConfig.deleteTopicEnable) + Assert.assertEquals(expectedConfig.compressionType, actualConfig.compressionType) + } + + private def atLeastXIntProp(x: Int): String = (nextInt(Int.MaxValue - 1) + x).toString + + private def atLeastOneIntProp: String = atLeastXIntProp(1) + + private def inRangeIntProp(fromInc: Int, toInc: Int): String = (nextInt(toInc + 1 - fromInc) + fromInc).toString + + @Test + def testFromPropsToProps() { + import scala.util.Random._ + val expected = new Properties() + KafkaConfig.configNames().foreach(name => { + name match { + case KafkaConfig.ZkConnectProp => expected.setProperty(name, "127.0.0.1:2181") + case KafkaConfig.ZkSessionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.ZkConnectionTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.ZkSyncTimeMsProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.NumNetworkThreadsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.NumIoThreadsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.BackgroundThreadsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.QueuedMaxRequestsProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.PortProp => expected.setProperty(name, "1234") + case KafkaConfig.HostNameProp => expected.setProperty(name, nextString(10)) + case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, nextString(10)) + case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321") + case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.MaxConnectionsPerIpProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.MaxConnectionsPerIpOverridesProp => expected.setProperty(name, "127.0.0.1:2, 127.0.0.2:3") + + case KafkaConfig.NumPartitionsProp => expected.setProperty(name, "2") + case KafkaConfig.LogDirsProp => expected.setProperty(name, "/tmp/logs,/tmp/logs2") + case KafkaConfig.LogDirProp => expected.setProperty(name, "/tmp/log") + case KafkaConfig.LogSegmentBytesProp => expected.setProperty(name, atLeastXIntProp(Message.MinHeaderSize)) + + case KafkaConfig.LogRollTimeMillisProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogRollTimeHoursProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.LogRetentionTimeMillisProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogRetentionTimeMinutesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogRetentionTimeHoursProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.LogCleanupIntervalMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.LogCleanupPolicyProp => expected.setProperty(name, randFrom(Defaults.Compact, Defaults.Delete)) + case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case KafkaConfig.LogCleanerMinCleanRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case KafkaConfig.LogCleanerEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.LogIndexSizeMaxBytesProp => expected.setProperty(name, atLeastXIntProp(4)) + case KafkaConfig.LogFlushIntervalMessagesProp => expected.setProperty(name, atLeastOneIntProp) + + case KafkaConfig.NumRecoveryThreadsPerDataDirProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.AutoCreateTopicsEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.MinInSyncReplicasProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.ControlledShutdownEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.OffsetsLoadBufferSizeProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsTopicPartitionsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsTopicSegmentBytesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsTopicCompressionCodecProp => expected.setProperty(name, randFrom(GZIPCompressionCodec.codec.toString, + SnappyCompressionCodec.codec.toString, LZ4CompressionCodec.codec.toString, NoCompressionCodec.codec.toString)) + case KafkaConfig.OffsetsRetentionMinutesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.OffsetCommitTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.DeleteTopicEnableProp => expected.setProperty(name, randFrom("true", "false")) + + // explicit, non trivial validations or with transient dependencies + + // require(brokerId >= -1 && brokerId <= maxReservedBrokerId) + case KafkaConfig.MaxReservedBrokerIdProp => expected.setProperty(name, "100") + case KafkaConfig.BrokerIdProp => expected.setProperty(name, inRangeIntProp(0, 100)) + // require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024) + case KafkaConfig.LogCleanerThreadsProp => expected.setProperty(name, "2") + case KafkaConfig.LogCleanerDedupeBufferSizeProp => expected.setProperty(name, (1024 * 1024 * 3 + 1).toString) + // require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs) + case KafkaConfig.ReplicaFetchWaitMaxMsProp => expected.setProperty(name, "321") + case KafkaConfig.ReplicaSocketTimeoutMsProp => expected.setProperty(name, atLeastXIntProp(321)) + // require(replicaFetchMaxBytes >= messageMaxBytes) + case KafkaConfig.MessageMaxBytesProp => expected.setProperty(name, "1234") + case KafkaConfig.ReplicaFetchMaxBytesProp => expected.setProperty(name, atLeastXIntProp(1234)) + // require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs) + case KafkaConfig.ReplicaLagTimeMaxMsProp => expected.setProperty(name, atLeastXIntProp(321)) + //require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor) + case KafkaConfig.OffsetCommitRequiredAcksProp => expected.setProperty(name, "-1") + case KafkaConfig.OffsetsTopicReplicationFactorProp => expected.setProperty(name, inRangeIntProp(-1, Short.MaxValue)) + //BrokerCompressionCodec.isValid(compressionType) + case KafkaConfig.CompressionTypeProp => expected.setProperty(name, randFrom(BrokerCompressionCodec.brokerCompressionOptions)) + + case nonNegativeIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) + } + }) + + val actual = KafkaConfig.fromProps(expected).toProps + Assert.assertEquals(expected, actual) + } + + @Test + def testFromPropsInvalid() { + def getBaseProperties(): Properties = { + val validRequiredProperties = new Properties() + validRequiredProperties.put(KafkaConfig.ZkConnectProp, "127.0.0.1") + validRequiredProperties + } + // to ensure a basis is valid - bootstraps all needed validation + KafkaConfig.fromProps(getBaseProperties()) + + KafkaConfig.configNames().foreach(name => { + name match { + case KafkaConfig.ZkConnectProp => // ignore string + case KafkaConfig.ZkSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ZkConnectionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ZkSyncTimeMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + + case KafkaConfig.BrokerIdProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.HostNameProp => // ignore string + case KafkaConfig.AdvertisedHostNameProp => //ignore string + case KafkaConfig.AdvertisedPortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.SocketSendBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.SocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.MaxConnectionsPerIpOverridesProp => + assertPropertyInvalid(getBaseProperties(), name, "127.0.0.1:not_a_number") + case KafkaConfig.ConnectionsMaxIdleMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + + case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogDirsProp => // ignore string + case KafkaConfig.LogDirProp => // ignore string + case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinHeaderSize - 1) + + case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.LogRetentionTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRetentionTimeMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogRetentionTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + + case KafkaConfig.LogRetentionBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanupIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogCleanupPolicyProp => assertPropertyInvalid(getBaseProperties(), name, "unknown_policy", "0") + case KafkaConfig.LogCleanerIoMaxBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerDedupeBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "1024") + case KafkaConfig.LogCleanerDedupeBufferLoadFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean") + case KafkaConfig.LogCleanerDeleteRetentionMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogCleanerMinCleanRatioProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogIndexSizeMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "3") + case KafkaConfig.LogFlushIntervalMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.LogFlushSchedulerIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LogFlushIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumRecoveryThreadsPerDataDirProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.AutoCreateTopicsEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.MinInSyncReplicasProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ControllerSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ControllerMessageQueueSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.DefaultReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaLagTimeMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaLagMaxMessagesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaSocketTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") + case KafkaConfig.ReplicaSocketReceiveBufferBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchWaitMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchMinBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.NumReplicaFetchersProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ProducerPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.AutoLeaderRebalanceEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.LeaderImbalancePerBrokerPercentageProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.LeaderImbalanceCheckIntervalSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.UncleanLeaderElectionEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsTopicCompressionCodecProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") + case KafkaConfig.OffsetsRetentionMinutesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") + + case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") + + case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") + } + }) + } + + @Test + def testSpecificProperties(): Unit = { + val defaults = new Properties() + defaults.put(KafkaConfig.ZkConnectProp, "127.0.0.1:2181") + // For ZkConnectionTimeoutMs + defaults.put(KafkaConfig.ZkSessionTimeoutMsProp, "1234") + defaults.put(KafkaConfig.MaxReservedBrokerIdProp, "1") + defaults.put(KafkaConfig.BrokerIdProp, "1") + defaults.put(KafkaConfig.HostNameProp, "127.0.0.1") + defaults.put(KafkaConfig.PortProp, "1122") + defaults.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, "127.0.0.1:2, 127.0.0.2:3") + defaults.put(KafkaConfig.LogDirProp, "/tmp1,/tmp2") + defaults.put(KafkaConfig.LogRollTimeHoursProp, "12") + defaults.put(KafkaConfig.LogRollTimeJitterHoursProp, "11") + defaults.put(KafkaConfig.LogRetentionTimeHoursProp, "10") + //For LogFlushIntervalMsProp + defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") + defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) + + val config = KafkaConfig.fromProps(defaults) + Assert.assertEquals("127.0.0.1:2181", config.zkConnect) + Assert.assertEquals(1234, config.zkConnectionTimeoutMs) + Assert.assertEquals(1, config.maxReservedBrokerId) + Assert.assertEquals(1, config.brokerId) + Assert.assertEquals("127.0.0.1", config.hostName) + Assert.assertEquals(1122, config.advertisedPort) + Assert.assertEquals("127.0.0.1", config.advertisedHostName) + Assert.assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides) + Assert.assertEquals(List("/tmp1", "/tmp2"), config.logDirs) + Assert.assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis) + Assert.assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis) + Assert.assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis) + Assert.assertEquals(123L, config.logFlushIntervalMs) + Assert.assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec) + } + + private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) { + values.foreach((value) => { + val props = validRequiredProps + props.setProperty(name, value.toString) + intercept[Exception] { + KafkaConfig.fromProps(props) + } + }) + } + + private def randFrom[T](choices: T*): T = { + import scala.util.Random + choices(Random.nextInt(choices.size)) + } + + private def randFrom[T](choices: List[T]): T = { + import scala.util.Random + choices(Random.nextInt(choices.size)) + } +} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 82dce80d553..7f47e6f9a74 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -17,6 +17,7 @@ package kafka.server +import org.apache.kafka.common.config.ConfigException import org.junit.Test import junit.framework.Assert._ import org.scalatest.junit.JUnit3Suite @@ -31,7 +32,7 @@ class KafkaConfigTest extends JUnit3Suite { val props = TestUtils.createBrokerConfig(0, 8181) props.put("log.retention.hours", "1") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis) } @@ -41,7 +42,7 @@ class KafkaConfigTest extends JUnit3Suite { val props = TestUtils.createBrokerConfig(0, 8181) props.put("log.retention.minutes", "30") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) } @@ -51,7 +52,7 @@ class KafkaConfigTest extends JUnit3Suite { val props = TestUtils.createBrokerConfig(0, 8181) props.put("log.retention.ms", "1800000") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) } @@ -60,7 +61,7 @@ class KafkaConfigTest extends JUnit3Suite { def testLogRetentionTimeNoConfigProvided() { val props = TestUtils.createBrokerConfig(0, 8181) - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRetentionTimeMillis) } @@ -71,7 +72,7 @@ class KafkaConfigTest extends JUnit3Suite { props.put("log.retention.minutes", "30") props.put("log.retention.hours", "1") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) } @@ -82,7 +83,7 @@ class KafkaConfigTest extends JUnit3Suite { props.put("log.retention.ms", "1800000") props.put("log.retention.minutes", "10") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) } @@ -95,7 +96,7 @@ class KafkaConfigTest extends JUnit3Suite { val props = TestUtils.createBrokerConfig(0, port) props.put("host.name", hostName) - val serverConfig = new KafkaConfig(props) + val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.advertisedHostName, hostName) assertEquals(serverConfig.advertisedPort, port) @@ -111,7 +112,7 @@ class KafkaConfigTest extends JUnit3Suite { props.put("advertised.host.name", advertisedHostName) props.put("advertised.port", advertisedPort.toString) - val serverConfig = new KafkaConfig(props) + val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.advertisedHostName, advertisedHostName) assertEquals(serverConfig.advertisedPort, advertisedPort) @@ -120,7 +121,7 @@ class KafkaConfigTest extends JUnit3Suite { @Test def testUncleanLeaderElectionDefault() { val props = TestUtils.createBrokerConfig(0, 8181) - val serverConfig = new KafkaConfig(props) + val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.uncleanLeaderElectionEnable, true) } @@ -129,7 +130,7 @@ class KafkaConfigTest extends JUnit3Suite { def testUncleanElectionDisabled() { val props = TestUtils.createBrokerConfig(0, 8181) props.put("unclean.leader.election.enable", String.valueOf(false)) - val serverConfig = new KafkaConfig(props) + val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.uncleanLeaderElectionEnable, false) } @@ -138,7 +139,7 @@ class KafkaConfigTest extends JUnit3Suite { def testUncleanElectionEnabled() { val props = TestUtils.createBrokerConfig(0, 8181) props.put("unclean.leader.election.enable", String.valueOf(true)) - val serverConfig = new KafkaConfig(props) + val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.uncleanLeaderElectionEnable, true) } @@ -148,8 +149,8 @@ class KafkaConfigTest extends JUnit3Suite { val props = TestUtils.createBrokerConfig(0, 8181) props.put("unclean.leader.election.enable", "invalid") - intercept[IllegalArgumentException] { - new KafkaConfig(props) + intercept[ConfigException] { + KafkaConfig.fromProps(props) } } @@ -158,7 +159,7 @@ class KafkaConfigTest extends JUnit3Suite { val props = TestUtils.createBrokerConfig(0, 8181) props.put("log.roll.ms", "1800000") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis) } @@ -169,7 +170,7 @@ class KafkaConfigTest extends JUnit3Suite { props.put("log.roll.ms", "1800000") props.put("log.roll.hours", "1") - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis) } @@ -178,7 +179,7 @@ class KafkaConfigTest extends JUnit3Suite { def testLogRollTimeNoConfigProvided() { val props = TestUtils.createBrokerConfig(0, 8181) - val cfg = new KafkaConfig(props) + val cfg = KafkaConfig.fromProps(props) assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis ) } @@ -186,7 +187,7 @@ class KafkaConfigTest extends JUnit3Suite { @Test def testDefaultCompressionType() { val props = TestUtils.createBrokerConfig(0, 8181) - val serverConfig = new KafkaConfig(props) + val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.compressionType, "producer") } @@ -195,7 +196,7 @@ class KafkaConfigTest extends JUnit3Suite { def testValidCompressionType() { val props = TestUtils.createBrokerConfig(0, 8181) props.put("compression.type", "gzip") - val serverConfig = new KafkaConfig(props) + val serverConfig = KafkaConfig.fromProps(props) assertEquals(serverConfig.compressionType, "gzip") } @@ -205,7 +206,7 @@ class KafkaConfigTest extends JUnit3Suite { val props = TestUtils.createBrokerConfig(0, 8181) props.put("compression.type", "abc") intercept[IllegalArgumentException] { - new KafkaConfig(props) + KafkaConfig.fromProps(props) } } } diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index c2ba07c5fdb..f2528059dd8 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -43,8 +43,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() // start both servers - val server1 = TestUtils.createServer(new KafkaConfig(configProps1)) - val server2 = TestUtils.createServer(new KafkaConfig(configProps2)) + val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1)) + val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2)) servers ++= List(server1, server2) } @@ -117,7 +117,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // start another controller val controllerId = 2 - val controllerConfig = new KafkaConfig(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) + val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.config.port)) val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index c06ee756bf0..8c9f9e748e8 100644 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -50,7 +50,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val logDirPath = config.getProperty("log.dir") logDir = new File(logDirPath) time = new MockTime() - server = TestUtils.createServer(new KafkaConfig(config), time) + server = TestUtils.createServer(KafkaConfig.fromProps(config), time) simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "") } diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index d5d351c4f25..92d6b2c672f 100644 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -16,6 +16,8 @@ */ package kafka.server +import java.util.Properties + import kafka.utils.TestUtils._ import kafka.utils.IntEncoder import kafka.utils.{Utils, TestUtils} @@ -31,12 +33,18 @@ import org.junit.Assert._ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { - val configs = TestUtils.createBrokerConfigs(2, false).map(new KafkaConfig(_) { - override val replicaLagTimeMaxMs = 5000L - override val replicaLagMaxMessages = 10L - override val replicaFetchWaitMaxMs = 1000 - override val replicaFetchMinBytes = 20 - }) + val replicaLagTimeMaxMs = 5000L + val replicaLagMaxMessages = 10L + val replicaFetchWaitMaxMs = 1000 + val replicaFetchMinBytes = 20 + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaLagMaxMessagesProp, replicaLagMaxMessages.toString) + overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchMinBytesProp, replicaFetchMinBytes.toString) + + val configs = TestUtils.createBrokerConfigs(2, false).map(KafkaConfig.fromProps(_, overridingProps)) val topic = "new-topic" val partitionId = 0 diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index a37a74dc89e..ea9b3150992 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -49,7 +49,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val logDirPath = config.getProperty("log.dir") logDir = new File(logDirPath) time = new MockTime() - server = TestUtils.createServer(new KafkaConfig(config), time) + server = TestUtils.createServer(KafkaConfig.fromProps(config), time) simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "test-client") val consumerMetadataRequest = ConsumerMetadataRequest(group) Stream.continually { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index da4bafc1e2a..1e64faf9a7e 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -28,7 +28,7 @@ import kafka.common._ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(2,false) - val configs = props.map(p => new KafkaConfig(p)) + val configs = props.map(p => KafkaConfig.fromProps(p)) var brokers: Seq[KafkaServer] = null val topic1 = "foo" val topic2 = "bar" diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index d1ed5c2c506..2849a5e7607 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -39,7 +39,7 @@ class ReplicaManagerTest extends JUnit3Suite { @Test def testHighWaterMarkDirectoryMapping() { val props = TestUtils.createBrokerConfig(1) - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) val zkClient = EasyMock.createMock(classOf[ZkClient]) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) val time: MockTime = new MockTime() @@ -56,7 +56,7 @@ class ReplicaManagerTest extends JUnit3Suite { def testHighwaterMarkRelativeDirectoryMapping() { val props = TestUtils.createBrokerConfig(1) props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) val zkClient = EasyMock.createMock(classOf[ZkClient]) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) val time: MockTime = new MockTime() @@ -72,7 +72,7 @@ class ReplicaManagerTest extends JUnit3Suite { @Test def testIllegalRequiredAcks() { val props = TestUtils.createBrokerConfig(1) - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) val zkClient = EasyMock.createMock(classOf[ZkClient]) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) val time: MockTime = new MockTime() diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala index cf2dd9455a9..96a8a5a8cb1 100644 --- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -25,9 +25,9 @@ import java.io.File class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { var props1 = TestUtils.createBrokerConfig(-1, TestUtils.choosePort) - var config1 = new KafkaConfig(props1) + var config1 = KafkaConfig.fromProps(props1) var props2 = TestUtils.createBrokerConfig(0, TestUtils.choosePort) - var config2 = new KafkaConfig(props2) + var config2 = KafkaConfig.fromProps(props2) val brokerMetaPropsFile = "meta.properties" @@ -52,7 +52,7 @@ class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { val server1 = new KafkaServer(config1) val server2 = new KafkaServer(config2) val props3 = TestUtils.createBrokerConfig(-1, TestUtils.choosePort) - val config3 = new KafkaConfig(props3) + val config3 = KafkaConfig.fromProps(props3) val server3 = new KafkaServer(config3) server1.startup() assertEquals(server1.config.brokerId,1001) @@ -78,7 +78,7 @@ class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { val logDirs = props1.getProperty("log.dir")+ "," + TestUtils.tempDir().getAbsolutePath + "," + TestUtils.tempDir().getAbsolutePath props1.setProperty("log.dir",logDirs) - config1 = new KafkaConfig(props1) + config1 = KafkaConfig.fromProps(props1) var server1 = new KafkaServer(config1) server1.startup() server1.shutdown() @@ -86,7 +86,7 @@ class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { // addition to log.dirs after generation of a broker.id from zk should be copied over val newLogDirs = props1.getProperty("log.dir") + "," + TestUtils.tempDir().getAbsolutePath props1.setProperty("log.dir",newLogDirs) - config1 = new KafkaConfig(props1) + config1 = KafkaConfig.fromProps(props1) server1 = new KafkaServer(config1) server1.startup() server1.shutdown() diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 82fa4cff450..b46daa43623 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -34,7 +34,7 @@ import junit.framework.Assert._ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val port = TestUtils.choosePort val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) + val config = KafkaConfig.fromProps(props) val host = "localhost" val topic = "test" @@ -105,7 +105,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { def testCleanShutdownWithDeleteTopicEnabled() { val newProps = TestUtils.createBrokerConfig(0, port) newProps.setProperty("delete.topic.enable", "true") - val newConfig = new KafkaConfig(newProps) + val newConfig = KafkaConfig.fromProps(newProps) val server = new KafkaServer(newConfig) server.startup() server.shutdown() @@ -118,7 +118,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { def testCleanShutdownAfterFailedStartup() { val newProps = TestUtils.createBrokerConfig(0, port) newProps.setProperty("zookeeper.connect", "fakehostthatwontresolve:65535") - val newConfig = new KafkaConfig(newProps) + val newConfig = KafkaConfig.fromProps(newProps) val server = new KafkaServer(newConfig) try { server.startup() diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 93af7dfcec7..60021ef8d8d 100644 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -33,7 +33,7 @@ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) val zooKeeperConnect = props.get("zookeeper.connect") props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot) - val server = TestUtils.createServer(new KafkaConfig(props)) + val server = TestUtils.createServer(KafkaConfig.fromProps(props)) val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot) assertTrue(pathExists) @@ -48,12 +48,12 @@ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { val brokerId = 0 val props1 = TestUtils.createBrokerConfig(brokerId) - val server1 = TestUtils.createServer(new KafkaConfig(props1)) + val server1 = TestUtils.createServer(KafkaConfig.fromProps(props1)) val brokerRegistration = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 val props2 = TestUtils.createBrokerConfig(brokerId) try { - TestUtils.createServer(new KafkaConfig(props2)) + TestUtils.createServer(KafkaConfig.fromProps(props2)) fail("Registering a broker with a conflicting id should fail") } catch { case e : RuntimeException => diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index fd8f32c4158..efb457334bd 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -24,7 +24,7 @@ import kafka.log.Log import kafka.message.{ByteBufferMessageSet, Message} import scala.Some -import java.util.Collections +import java.util.{Properties, Collections} import java.util.concurrent.atomic.AtomicBoolean import collection.JavaConversions._ @@ -35,11 +35,16 @@ import junit.framework.Assert._ class SimpleFetchTest extends JUnit3Suite { - val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_) { - override val replicaLagTimeMaxMs = 100L - override val replicaFetchWaitMaxMs = 100 - override val replicaLagMaxMessages = 10L - }) + val replicaLagTimeMaxMs = 100L + val replicaFetchWaitMaxMs = 100 + val replicaLagMaxMessages = 10L + + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, replicaLagTimeMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaFetchWaitMaxMsProp, replicaFetchWaitMaxMs.toString) + overridingProps.put(KafkaConfig.ReplicaLagMaxMessagesProp, replicaLagMaxMessages.toString) + + val configs = TestUtils.createBrokerConfigs(2).map(KafkaConfig.fromProps(_, overridingProps)) // set the replica manager with the partition val time = new MockTime diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index 305498adf41..2edc814e7a9 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -53,7 +53,7 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { @Test def testUpdateLeaderAndIsr() { - val configs = TestUtils.createBrokerConfigs(1).map(new KafkaConfig(_)) + val configs = TestUtils.createBrokerConfigs(1).map(KafkaConfig.fromProps) val log = EasyMock.createMock(classOf[kafka.log.Log]) EasyMock.expect(log.logEndOffset).andReturn(20).anyTimes() EasyMock.expect(log)