This is an automated email from the ASF dual-hosted git repository.

mjsax 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 d233eb98f7c KAFKA-14957: Update-Description-String (#13909)
d233eb98f7c is described below

commit d233eb98f7c7e55fe0dd673dbc058ddf619663a7
Author: Owen Leung <owen.leu...@gmail.com>
AuthorDate: Sun Feb 11 04:46:51 2024 +0800

    KAFKA-14957: Update-Description-String (#13909)
    
    HTML code for configs is auto-generated and for Kafka Streams config 
`state.dir` produces a confusing default value.
    This PR adds a new property `alternativeString` to set a "default" value 
which should be rendered in HTML instead of the actual default value.
    
    Reviewers: Manyanda Chitimbo <manyanda.chiti...@gmail.com>, @eziosudo 
<ezios...@gmail.com>, Matthias J. Sax <matth...@confluent.io>
---
 .../org/apache/kafka/common/config/ConfigDef.java  | 55 ++++++++++++++++++++--
 .../apache/kafka/common/config/ConfigDefTest.java  |  2 +-
 .../kafka/connect/runtime/AbstractHerderTest.java  |  2 +-
 .../org/apache/kafka/streams/StreamsConfig.java    |  5 +-
 4 files changed, 55 insertions(+), 9 deletions(-)

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 6f2e7413ccc..57df493347b 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
@@ -152,7 +152,30 @@ public class ConfigDef {
      */
     public ConfigDef define(String name, Type type, Object defaultValue, 
Validator validator, Importance importance, String documentation,
                             String group, int orderInGroup, Width width, 
String displayName, List<String> dependents, Recommender recommender) {
-        return define(new ConfigKey(name, type, defaultValue, validator, 
importance, documentation, group, orderInGroup, width, displayName, dependents, 
recommender, false));
+        return define(new ConfigKey(name, type, defaultValue, validator, 
importance, documentation, group, orderInGroup, width, displayName, dependents, 
recommender, false, null));
+    }
+
+    /**
+     * 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          the validator to use in checking the 
correctness of the config
+     * @param importance         the importance of this config
+     * @param documentation      the documentation string for the config
+     * @param group              the group this config belongs to
+     * @param orderInGroup       the order of this config in the group
+     * @param width              the width of the config
+     * @param displayName        the name suitable for display
+     * @param dependents         the configurations that are dependents of 
this configuration
+     * @param recommender        the recommender provides valid values given 
the parent configuration values
+     * @param alternativeString  the string which will be used to override the 
string of defaultValue
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, 
Validator validator, Importance importance, String documentation,
+                            String group, int orderInGroup, Width width, 
String displayName, List<String> dependents, Recommender recommender,
+                            String alternativeString) {
+        return define(new ConfigKey(name, type, defaultValue, validator, 
importance, documentation, group, orderInGroup, width, displayName, dependents, 
recommender, false, alternativeString));
     }
 
     /**
@@ -389,6 +412,21 @@ public class ConfigDef {
         return define(name, type, defaultValue, null, importance, 
documentation);
     }
 
+    /**
+     * 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 documentation     The documentation string for the config
+     * @param alternativeString The string which will be used to override the 
string of defaultValue
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, 
Importance importance, String documentation, String alternativeString) {
+        return define(name, type, defaultValue, null, importance, 
documentation, null, -1, Width.NONE,
+                name, Collections.emptyList(), null, alternativeString);
+    }
+
     /**
      * Define a new configuration with no default value and no special 
validation logic
      * @param name          The name of the config parameter
@@ -411,7 +449,7 @@ public class ConfigDef {
      * @return This ConfigDef so you can chain calls
      */
     public ConfigDef defineInternal(final String name, final Type type, final 
Object defaultValue, final Importance importance) {
-        return define(new ConfigKey(name, type, defaultValue, null, 
importance, "", "", -1, Width.NONE, name, Collections.emptyList(), null, true));
+        return define(new ConfigKey(name, type, defaultValue, null, 
importance, "", "", -1, Width.NONE, name, Collections.emptyList(), null, true, 
null));
     }
 
     /**
@@ -426,7 +464,7 @@ public class ConfigDef {
      * @return This ConfigDef so you can chain calls
      */
     public ConfigDef defineInternal(final String name, final Type type, final 
Object defaultValue, final Validator validator, final Importance importance, 
final String documentation) {
-        return define(new ConfigKey(name, type, defaultValue, validator, 
importance, documentation, "", -1, Width.NONE, name, Collections.emptyList(), 
null, true));
+        return define(new ConfigKey(name, type, defaultValue, validator, 
importance, documentation, "", -1, Width.NONE, name, Collections.emptyList(), 
null, true, null));
     }
 
     /**
@@ -1216,12 +1254,13 @@ public class ConfigDef {
         public final List<String> dependents;
         public final Recommender recommender;
         public final boolean internalConfig;
+        public final String alternativeString;
 
         public ConfigKey(String name, Type type, Object defaultValue, 
Validator validator,
                          Importance importance, String documentation, String 
group,
                          int orderInGroup, Width width, String displayName,
                          List<String> dependents, Recommender recommender,
-                         boolean internalConfig) {
+                         boolean internalConfig, String alternativeString) {
             this.name = name;
             this.type = type;
             boolean hasDefault = !NO_DEFAULT_VALUE.equals(defaultValue);
@@ -1238,6 +1277,7 @@ public class ConfigDef {
             this.displayName = displayName;
             this.recommender = recommender;
             this.internalConfig = internalConfig;
+            this.alternativeString = alternativeString;
         }
 
         public boolean hasDefault() {
@@ -1530,7 +1570,8 @@ public class ConfigDef {
                     key.displayName,
                     embeddedDependents(keyPrefix, key.dependents),
                     embeddedRecommender(keyPrefix, key.recommender),
-                    key.internalConfig));
+                    key.internalConfig,
+                    key.alternativeString));
         }
     }
 
@@ -1652,6 +1693,10 @@ public class ConfigDef {
                     "<tbody>\n");
             for (String detail : headers()) {
                 if (detail.equals("Name") || detail.equals("Description")) 
continue;
+                if (detail.equals("Default") && key.alternativeString != null) 
{
+                    addConfigDetail(b, detail, key.alternativeString);
+                    continue;
+                }
                 addConfigDetail(b, detail, getConfigValue(key, detail));
             }
             if (hasUpdateModes) {
diff --git 
a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java 
b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
index d2c572f06f4..890cbb52a66 100644
--- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
@@ -95,7 +95,7 @@ public class ConfigDefTest {
 
     @Test
     public void testNullDefault() {
-        ConfigDef def = new ConfigDef().define("a", Type.INT, null, null, 
null, "docs");
+        ConfigDef def = new ConfigDef().define("a", Type.INT, null, null, 
"docs");
         Map<String, Object> vals = def.parse(new Properties());
 
         assertNull(vals.get("a"));
diff --git 
a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
 
b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
index 7c39ce536be..a11b0232d89 100644
--- 
a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
+++ 
b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
@@ -1119,7 +1119,7 @@ public class AbstractHerderTest {
     protected void addConfigKey(Map<String, ConfigDef.ConfigKey> keys, String 
name, String group) {
         keys.put(name, new ConfigDef.ConfigKey(name, ConfigDef.Type.STRING, 
null, null,
                 ConfigDef.Importance.HIGH, "doc", group, 10,
-                ConfigDef.Width.MEDIUM, "display name", 
Collections.emptyList(), null, false));
+                ConfigDef.Width.MEDIUM, "display name", 
Collections.emptyList(), null, false, null));
     }
 
     protected void addValue(List<ConfigValue> values, String name, String 
value, String...errors) {
diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java 
b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index 2c8a1b59ff9..70520c1eee2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -749,7 +749,7 @@ public class StreamsConfig extends AbstractConfig {
     /** {@code state.dir} */
     @SuppressWarnings("WeakerAccess")
     public static final String STATE_DIR_CONFIG = "state.dir";
-    private static final String STATE_DIR_DOC = "Directory location for state 
store. This path must be unique for each streams instance sharing the same 
underlying filesystem.";
+    private static final String STATE_DIR_DOC = "Directory location for state 
store. This path must be unique for each streams instance sharing the same 
underlying filesystem. Note that if not configured, then the default location 
will be different in each environment as it is computed using 
System.getProperty(\"java.io.tmpdir\")";
 
     /** {@code task.timeout.ms} */
     public static final String TASK_TIMEOUT_MS_CONFIG = "task.timeout.ms";
@@ -852,7 +852,8 @@ public class StreamsConfig extends AbstractConfig {
                     Type.STRING,
                     System.getProperty("java.io.tmpdir") + File.separator + 
"kafka-streams",
                     Importance.HIGH,
-                    STATE_DIR_DOC)
+                    STATE_DIR_DOC,
+                    "${java.io.tmpdir}")
 
             // MEDIUM
 

Reply via email to