C0urante commented on code in PR #13168:
URL: https://github.com/apache/kafka/pull/13168#discussion_r1107747315


##########
clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java:
##########
@@ -389,22 +389,27 @@ public void logUnused() {
     private <T> T getConfiguredInstance(Object klass, Class<T> t, Map<String, 
Object> configPairs) {
         if (klass == null)
             return null;
+        Object o = null;
+        try {
+            if (klass instanceof String) {
+                try {
+                    o = Utils.newInstance((String) klass, t);
+                } catch (ClassNotFoundException e) {
+                    throw new KafkaException("Class " + klass + " cannot be 
found", e);
+                }
+            } else if (klass instanceof Class<?>) {
+                o = Utils.newInstance((Class<?>) klass);
+            } else
+                throw new KafkaException("Unexpected element of type " + 
klass.getClass().getName() + ", expected String or Class");
+            if (!t.isInstance(o))
+                throw new KafkaException(klass + " is not an instance of " + 
t.getName());
+            if (o instanceof Configurable)
+                ((Configurable) o).configure(configPairs);
+        } catch (Exception e) {
+            maybeClose(o, "AutoCloseable object constructed and configured 
during failed call to getConfiguredInstance");
+            throw e;
+        }
 

Review Comment:
   Nit: remove this extra blank line?



##########
clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java:
##########
@@ -599,6 +624,31 @@ public TestConfig(Map<?, ?> props) {
         }
     }
 
+    private static class InterceptorTestConfig extends AbstractConfig {
+        private final int targetInterceptor = 3;
+        private static final ConfigDef CONFIG;
+        private static final String INTERCEPTOR_CLASSES_CONFIG_DOC = "A list 
of classes to use as interceptors.";
+
+        public static final String INTERCEPTOR_CLASSES_CONFIG = 
"interceptor.classes";
+        public static final String CLIENT_ID_CONFIG = "client.id";
+        public static final String BOOTSTRAP_SERVERS_CONFIG = 
"bootstrap.servers";
+
+        static {
+            CONFIG = new ConfigDef().define(INTERCEPTOR_CLASSES_CONFIG,
+                    Type.LIST,
+                    Collections.emptyList(),
+                    Importance.LOW,
+                    INTERCEPTOR_CLASSES_CONFIG_DOC);
+        }
+        public InterceptorTestConfig(Map<?, ?> props) {
+            super(CONFIG, props);
+        }
+
+        public int getTargetInterceptor() {
+            return targetInterceptor;
+        }
+    }

Review Comment:
   Sorry, I'm still not in favor of this additional class. The name of the 
config property isn't super critical and it's not necessary to try to 
instantiate metrics reporters when using the `metric.reporters` property in the 
test config; this is only constructed for testing, after all.
   
   Please remove this class and simplify the testing logic by using the 
existing `TestConfig` class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to