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

chia7712 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 db56553fd3b KAFKA-17530 Remove blacklist/whitelist from ReplaceField 
(#17237)
db56553fd3b is described below

commit db56553fd3b1f265788e3a97ce6f537af00fdb75
Author: TaiJuWu <[email protected]>
AuthorDate: Wed Sep 25 01:02:28 2024 +0800

    KAFKA-17530 Remove blacklist/whitelist from ReplaceField (#17237)
    
    Reviewers: Chia-Ping Tsai <[email protected]>
---
 .../kafka/connect/transforms/ReplaceField.java     | 11 +-----
 .../kafka/connect/transforms/ReplaceFieldTest.java | 40 ----------------------
 docs/upgrade.html                                  |  5 ++-
 3 files changed, 5 insertions(+), 51 deletions(-)

diff --git 
a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
 
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
index 9364f08c6d1..6f337e4632a 100644
--- 
a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
+++ 
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
@@ -20,10 +20,8 @@ import org.apache.kafka.common.cache.Cache;
 import org.apache.kafka.common.cache.LRUCache;
 import org.apache.kafka.common.cache.SynchronizedCache;
 import org.apache.kafka.common.config.ConfigDef;
-import org.apache.kafka.common.config.ConfigDef.Importance;
 import org.apache.kafka.common.config.ConfigException;
 import org.apache.kafka.common.utils.AppInfoParser;
-import org.apache.kafka.common.utils.ConfigUtils;
 import org.apache.kafka.connect.components.Versioned;
 import org.apache.kafka.connect.connector.ConnectRecord;
 import org.apache.kafka.connect.data.Field;
@@ -60,12 +58,8 @@ public abstract class ReplaceField<R extends 
ConnectRecord<R>> implements Transf
     public static final ConfigDef CONFIG_DEF = new ConfigDef()
             .define(ConfigName.EXCLUDE, ConfigDef.Type.LIST, 
Collections.emptyList(), ConfigDef.Importance.MEDIUM,
                     "Fields to exclude. This takes precedence over the fields 
to include.")
-            .define("blacklist", ConfigDef.Type.LIST, null, Importance.LOW,
-                    "Deprecated. Use " + ConfigName.EXCLUDE + " instead.")
             .define(ConfigName.INCLUDE, ConfigDef.Type.LIST, 
Collections.emptyList(), ConfigDef.Importance.MEDIUM,
                     "Fields to include. If specified, only these fields will 
be used.")
-            .define("whitelist", ConfigDef.Type.LIST, null, Importance.LOW,
-                    "Deprecated. Use " + ConfigName.INCLUDE + " instead.")
             .define(ConfigName.RENAME, ConfigDef.Type.LIST, 
Collections.emptyList(),
                 ConfigDef.LambdaValidator.with(
                     (name, value) -> {
@@ -95,10 +89,7 @@ public abstract class ReplaceField<R extends 
ConnectRecord<R>> implements Transf
 
     @Override
     public void configure(Map<String, ?> configs) {
-        final SimpleConfig config = new SimpleConfig(CONFIG_DEF, 
ConfigUtils.translateDeprecatedConfigs(configs, new String[][]{
-            {ConfigName.INCLUDE, "whitelist"},
-            {ConfigName.EXCLUDE, "blacklist"},
-        }));
+        final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs);
 
         exclude = new HashSet<>(config.getList(ConfigName.EXCLUDE));
         include = new HashSet<>(config.getList(ConfigName.INCLUDE));
diff --git 
a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java
 
b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java
index 31c24ca5b8a..e0f17e065dd 100644
--- 
a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java
+++ 
b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java
@@ -144,46 +144,6 @@ public class ReplaceFieldTest {
         assertEquals(true, updatedValue.getBoolean("bar"));
     }
 
-    @Test
-    public void testIncludeBackwardsCompatibility() {
-        final Map<String, String> props = new HashMap<>();
-        props.put("whitelist", "abc,foo");
-        props.put("renames", "abc:xyz,foo:bar");
-
-        xform.configure(props);
-
-        final SinkRecord record = new SinkRecord("test", 0, null, null, null, 
null, 0);
-        final SinkRecord transformedRecord = xform.apply(record);
-
-        assertNull(transformedRecord.value());
-        assertNull(transformedRecord.valueSchema());
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testExcludeBackwardsCompatibility() {
-        final Map<String, String> props = new HashMap<>();
-        props.put("blacklist", "dont");
-        props.put("renames", "abc:xyz,foo:bar");
-
-        xform.configure(props);
-
-        final Map<String, Object> value = new HashMap<>();
-        value.put("dont", "whatever");
-        value.put("abc", 42);
-        value.put("foo", true);
-        value.put("etc", "etc");
-
-        final SinkRecord record = new SinkRecord("test", 0, null, null, null, 
value, 0);
-        final SinkRecord transformedRecord = xform.apply(record);
-
-        final Map<String, Object> updatedValue = (Map<String, Object>) 
transformedRecord.value();
-        assertEquals(3, updatedValue.size());
-        assertEquals(42, updatedValue.get("xyz"));
-        assertEquals(true, updatedValue.get("bar"));
-        assertEquals("etc", updatedValue.get("etc"));
-    }
-
     @Test
     public void testReplaceFieldVersionRetrievedFromAppInfoParser() {
         assertEquals(AppInfoParser.getVersion(), xform.version());
diff --git a/docs/upgrade.html b/docs/upgrade.html
index b079263bcbf..5c0f8b8ec16 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -51,6 +51,10 @@
                 The <code>--whitelist</code> option was removed from the 
<code>kafka-console-consumer</code> command line tool.
                 Please use <code>--include</code> instead.
             </li>
+            <li>
+                The <code>--whitelist</code> and <code>--blacklist</code> 
options were removed from the 
<code>org.apache.kafka.connect.transforms.ReplaceField</code>.
+                Please use <code>--include</code> and <code>--exclude</code> 
instead.
+            </li>
             <li>
                 The <code>metrics.jmx.blacklist</code> was removed from the 
<code>org.apache.kafka.common.metrics.JmxReporter</code>
                 Please use <code>metrics.jmx.exclude</code> instead.
@@ -59,7 +63,6 @@
                 The <code>metrics.jmx.whitelist</code> was removed from the 
<code>org.apache.kafka.common.metrics.JmxReporter</code>
                 Please use <code>metrics.jmx.include</code> instead.
             </li>
-
         </ul>
     </ul>
 

Reply via email to