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


##########
clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java:
##########
@@ -203,4 +205,15 @@ public static Map<String, Object> 
postProcessReconnectBackoffConfigs(AbstractCon
         }
         return rval;
     }
+
+    public static void postValidateSaslMechanismConfig(AbstractConfig config) {
+        SecurityProtocol securityProtocol = 
SecurityProtocol.forName(config.getString(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG));
+        String clientSaslMechanism = 
config.getString(SaslConfigs.SASL_MECHANISM);
+        if (securityProtocol == SecurityProtocol.SASL_PLAINTEXT || 
securityProtocol == SecurityProtocol.SASL_SSL) {
+            if (clientSaslMechanism == null || 
clientSaslMechanism.trim().isEmpty()) {
+                throw new ConfigException(SaslConfigs.SASL_MECHANISM, null, 
"When the " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG +
+                        " configuration enables the SASL mechanism, mechanism 
must be non-null or non-empty string.");

Review Comment:
   Nit:
   
   ```suggestion
                           " configuration enables SASL, mechanism must be 
non-null and non-empty");
   ```



##########
clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java:
##########
@@ -220,6 +223,7 @@ public class AdminClientConfig extends AbstractConfig {
 
     @Override
     protected Map<String, Object> postProcessParsedConfig(final Map<String, 
Object> parsedValues) {
+        CommonClientConfigs.postValidateSaslMechanismConfig(this);

Review Comment:
   This is pretty smooth--you've only added one line to an existing method in 
this class, and we now get intelligent, conditional validation of the SASL 
mechanism config. Nice job!



##########
clients/src/test/java/org/apache/kafka/clients/admin/AdminClientConfigTest.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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 org.apache.kafka.clients.admin;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class AdminClientConfigTest {
+
+    @Test
+    public void testInvalidSaslMechanism() {
+        Map<String, Object> configs = new HashMap<>();
+        configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, 
SecurityProtocol.SASL_PLAINTEXT.name);
+        configs.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:8121");
+        configs.put(SaslConfigs.SASL_MECHANISM, null);
+        ConfigException ce = assertThrows(ConfigException.class, () -> new 
AdminClientConfig(configs));
+        assertTrue(ce.getMessage().contains(SaslConfigs.SASL_MECHANISM));
+
+        configs.put(SaslConfigs.SASL_MECHANISM, "");
+        ce = assertThrows(ConfigException.class, () -> new 
AdminClientConfig(configs));
+        assertTrue(ce.getMessage().contains(SaslConfigs.SASL_MECHANISM));
+
+        configs.put(SaslConfigs.SASL_MECHANISM, " ");
+        ce = assertThrows(ConfigException.class, () -> new 
AdminClientConfig(configs));
+        assertTrue(ce.getMessage().contains(SaslConfigs.SASL_MECHANISM));

Review Comment:
   Nit: I don't think it's worth adding this test case. At this point we're 
really verifying the behavior of the `AbstractConfig` and `ConfigDef` classes 
and how they automatically trim strings, not any special logic in the 
`AdminClient` class itself.



##########
clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java:
##########
@@ -203,4 +205,15 @@ public static Map<String, Object> 
postProcessReconnectBackoffConfigs(AbstractCon
         }
         return rval;
     }
+
+    public static void postValidateSaslMechanismConfig(AbstractConfig config) {
+        SecurityProtocol securityProtocol = 
SecurityProtocol.forName(config.getString(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG));
+        String clientSaslMechanism = 
config.getString(SaslConfigs.SASL_MECHANISM);
+        if (securityProtocol == SecurityProtocol.SASL_PLAINTEXT || 
securityProtocol == SecurityProtocol.SASL_SSL) {
+            if (clientSaslMechanism == null || 
clientSaslMechanism.trim().isEmpty()) {

Review Comment:
   Nit: We don't need to `trim` here, do we?
   ```suggestion
               if (clientSaslMechanism == null || 
clientSaslMechanism.isEmpty()) {
   ```



-- 
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