You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ij...@apache.org on 2019/08/04 06:59:25 UTC
[kafka] branch trunk updated: MINOR: Fix potential bug in
LogConfig.getConfigValue and improve test coverage (#7159)
This is an automated email from the ASF dual-hosted git repository.
ijuma pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new f70ece2 MINOR: Fix potential bug in LogConfig.getConfigValue and improve test coverage (#7159)
f70ece2 is described below
commit f70ece26d1cce23556f8f69ca2ccb2bb9e4f4de1
Author: Ismael Juma <is...@juma.me.uk>
AuthorDate: Sat Aug 3 23:59:06 2019 -0700
MINOR: Fix potential bug in LogConfig.getConfigValue and improve test coverage (#7159)
LogConfig.getConfigValue would throw a NoSuchElementException if any log
config was defined without a server default mapping.
Added a unit test for `getConfigValue` and a sanity test for
`toHtml`/`toRst`/`toEnrichedRst`, which were previously not exercised during
the test suite.
Reviewers: Jason Gustafson <ja...@confluent.io>, José Armando García Sancio <js...@users.noreply.github.com>
---
.../kafka/clients/producer/ProducerConfig.java | 2 +-
core/src/main/scala/kafka/log/LogConfig.scala | 18 +++++++--
.../test/scala/unit/kafka/log/LogConfigTest.scala | 46 +++++++++++++++++++++-
3 files changed, 60 insertions(+), 6 deletions(-)
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
index 4a4aeba..0d87284 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
@@ -405,7 +405,7 @@ public class ProducerConfig extends AbstractConfig {
}
public static ConfigDef configDef() {
- return new ConfigDef(CONFIG);
+ return new ConfigDef(CONFIG);
}
public static void main(String[] args) {
diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala
index bcc3f12..c61e023 100755
--- a/core/src/main/scala/kafka/log/LogConfig.scala
+++ b/core/src/main/scala/kafka/log/LogConfig.scala
@@ -181,9 +181,17 @@ object LogConfig {
"[PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:... or alternatively the wildcard '*' can be used to throttle " +
"all replicas for this topic."
- private class LogConfigDef extends ConfigDef {
+ private[log] val ServerDefaultHeaderName = "Server Default Property"
+
+ // Package private for testing
+ private[log] class LogConfigDef(base: ConfigDef) extends ConfigDef(base) {
+ def this() = this(new ConfigDef)
private final val serverDefaultConfigNames = mutable.Map[String, String]()
+ base match {
+ case b: LogConfigDef => serverDefaultConfigNames ++= b.serverDefaultConfigNames
+ case _ =>
+ }
def define(name: String, defType: ConfigDef.Type, defaultValue: Any, validator: Validator,
importance: ConfigDef.Importance, doc: String, serverDefaultConfigName: String): LogConfigDef = {
@@ -206,11 +214,12 @@ object LogConfig {
this
}
- override def headers = List("Name", "Description", "Type", "Default", "Valid Values", "Server Default Property", "Importance").asJava
+ override def headers = List("Name", "Description", "Type", "Default", "Valid Values", ServerDefaultHeaderName,
+ "Importance").asJava
override def getConfigValue(key: ConfigKey, headerName: String): String = {
headerName match {
- case "Server Default Property" => serverDefaultConfigNames.get(key.name).get
+ case ServerDefaultHeaderName => serverDefaultConfigNames.getOrElse(key.name, null)
case _ => super.getConfigValue(key, headerName)
}
}
@@ -218,6 +227,9 @@ object LogConfig {
def serverConfigName(configName: String): Option[String] = serverDefaultConfigNames.get(configName)
}
+ // Package private for testing, return a copy since it's a mutable global variable
+ private[log] def configDefCopy: LogConfigDef = new LogConfigDef(configDef)
+
private val configDef: LogConfigDef = {
import org.apache.kafka.common.config.ConfigDef.Importance._
import org.apache.kafka.common.config.ConfigDef.Range._
diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
index 2cd7904..6f3332f 100644
--- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
@@ -19,9 +19,11 @@ package kafka.log
import java.util.Properties
-import kafka.server.{ThrottledReplicaListValidator, KafkaConfig, KafkaServer}
+import kafka.server.{KafkaConfig, KafkaServer, ThrottledReplicaListValidator}
import kafka.utils.TestUtils
-import org.apache.kafka.common.config.ConfigException
+import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM
+import org.apache.kafka.common.config.ConfigDef.Type.INT
+import org.apache.kafka.common.config.{ConfigException, TopicConfig}
import org.junit.{Assert, Test}
import org.junit.Assert._
import org.scalatest.Assertions._
@@ -113,6 +115,46 @@ class LogConfigTest {
assertFalse(isValid("100:0,10 : "))
}
+ /* Sanity check that toHtml produces one of the expected configs */
+ @Test
+ def testToHtml(): Unit = {
+ val html = LogConfig.configDefCopy.toHtmlTable
+ val expectedConfig = "<td>file.delete.delay.ms</td>"
+ assertTrue(s"Could not find `$expectedConfig` in:\n $html", html.contains(expectedConfig))
+ }
+
+ /* Sanity check that toEnrichedRst produces one of the expected configs */
+ @Test
+ def testToEnrichedRst(): Unit = {
+ val rst = LogConfig.configDefCopy.toEnrichedRst
+ val expectedConfig = "``file.delete.delay.ms``"
+ assertTrue(s"Could not find `$expectedConfig` in:\n $rst", rst.contains(expectedConfig))
+ }
+
+ /* Sanity check that toEnrichedRst produces one of the expected configs */
+ @Test
+ def testToRst(): Unit = {
+ val rst = LogConfig.configDefCopy.toRst
+ val expectedConfig = "``file.delete.delay.ms``"
+ assertTrue(s"Could not find `$expectedConfig` in:\n $rst", rst.contains(expectedConfig))
+ }
+
+ @Test
+ def testGetConfigValue(): Unit = {
+ // Add a config that doesn't set the `serverDefaultConfigName`
+ val configDef = LogConfig.configDefCopy
+ val configNameWithNoServerMapping = "log.foo"
+ configDef.define(configNameWithNoServerMapping, INT, 1, MEDIUM, s"$configNameWithNoServerMapping doc")
+
+ val deleteDelayKey = configDef.configKeys.get(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG)
+ val deleteDelayServerDefault = configDef.getConfigValue(deleteDelayKey, LogConfig.ServerDefaultHeaderName)
+ assertEquals(KafkaConfig.LogDeleteDelayMsProp, deleteDelayServerDefault)
+
+ val keyWithNoServerMapping = configDef.configKeys.get(configNameWithNoServerMapping)
+ val nullServerDefault = configDef.getConfigValue(keyWithNoServerMapping, LogConfig.ServerDefaultHeaderName)
+ assertNull(nullServerDefault)
+ }
+
private def isValid(configValue: String): Boolean = {
try {
ThrottledReplicaListValidator.ensureValidString("", configValue)