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 9f78771a1f5 KAFKA-18693 Remove PasswordEncoder (#18790)
9f78771a1f5 is described below

commit 9f78771a1f576af39f2bd71926fe8127799a4af7
Author: Ming-Yen Chung <[email protected]>
AuthorDate: Tue Feb 4 13:18:41 2025 +0800

    KAFKA-18693 Remove PasswordEncoder (#18790)
    
    Reviewers: Chia-Ping Tsai <[email protected]>
---
 .../scala/kafka/server/DynamicBrokerConfig.scala   | 40 ---------------------
 .../kafka/server/DynamicBrokerConfigTest.scala     |  2 +-
 .../org/apache/kafka/security/PasswordEncoder.java | 42 ----------------------
 3 files changed, 1 insertion(+), 83 deletions(-)

diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala 
b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index 5f60a07bcd2..5887bd98c1b 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -30,13 +30,11 @@ import org.apache.kafka.common.Reconfigurable
 import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
 import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, 
ConfigException, SaslConfigs, SslConfigs}
 import org.apache.kafka.common.metrics.{Metrics, MetricsReporter}
-import org.apache.kafka.common.config.types.Password
 import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable}
 import org.apache.kafka.common.security.authenticator.LoginManager
 import org.apache.kafka.common.utils.{ConfigUtils, Utils}
 import org.apache.kafka.coordinator.transaction.TransactionLogConfig
 import org.apache.kafka.network.SocketServerConfigs
-import org.apache.kafka.security.PasswordEncoder
 import org.apache.kafka.server.ProcessRole
 import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, 
ServerLogConfigs, ServerTopicConfigSynonyms}
 import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
@@ -210,7 +208,6 @@ class DynamicBrokerConfig(private val kafkaConfig: 
KafkaConfig) extends Logging
   private val lock = new ReentrantReadWriteLock
   private var metricsReceiverPluginOpt: Option[ClientMetricsReceiverPlugin] = _
   private var currentConfig: KafkaConfig = _
-  private val dynamicConfigPasswordEncoder = Some(PasswordEncoder.NOOP)
 
   private[server] def initialize(clientMetricsReceiverPluginOpt: 
Option[ClientMetricsReceiverPlugin]): Unit = {
     currentConfig = new KafkaConfig(kafkaConfig.props, false)
@@ -358,27 +355,6 @@ class DynamicBrokerConfig(private val kafkaConfig: 
KafkaConfig) extends Logging
     })
   }
 
-  private def passwordEncoder: PasswordEncoder = {
-    dynamicConfigPasswordEncoder.getOrElse(throw new ConfigException("Password 
encoder secret not configured"))
-  }
-
-  private[server] def toPersistentProps(configProps: Properties, 
perBrokerConfig: Boolean): Properties = {
-    val props = configProps.clone().asInstanceOf[Properties]
-
-    def encodePassword(configName: String, value: String): Unit = {
-      if (value != null) {
-        if (!perBrokerConfig)
-          throw new ConfigException("Password config can be defined only at 
broker level")
-        props.setProperty(configName, passwordEncoder.encode(new 
Password(value)))
-      }
-    }
-    configProps.asScala.foreachEntry { (name, value) =>
-      if (isPasswordConfig(name))
-        encodePassword(name, value)
-    }
-    props
-  }
-
   private[server] def fromPersistentProps(persistentProps: Properties,
                                           perBrokerConfig: Boolean): 
Properties = {
     val props = persistentProps.clone().asInstanceOf[Properties]
@@ -397,22 +373,6 @@ class DynamicBrokerConfig(private val kafkaConfig: 
KafkaConfig) extends Logging
     if (!perBrokerConfig)
       removeInvalidProps(perBrokerConfigs(props), "Per-broker configs defined 
at default cluster level will be ignored")
 
-    def decodePassword(configName: String, value: String): Unit = {
-      if (value != null) {
-        try {
-          props.setProperty(configName, passwordEncoder.decode(value).value)
-        } catch {
-          case e: Exception =>
-            error(s"Dynamic password config $configName could not be decoded, 
ignoring.", e)
-            props.remove(configName)
-        }
-      }
-    }
-
-    props.asScala.foreachEntry { (name, value) =>
-      if (isPasswordConfig(name))
-        decodePassword(name, value)
-    }
     props
   }
 
diff --git 
a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala 
b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
index e8702eacfa3..7abe369b6d9 100755
--- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
@@ -377,7 +377,7 @@ class DynamicBrokerConfigTest {
 
     def updateConfig(): Unit = {
       if (perBrokerConfig)
-        config.dynamicConfig.updateBrokerConfig(0, 
config.dynamicConfig.toPersistentProps(props, perBrokerConfig))
+        config.dynamicConfig.updateBrokerConfig(0, props)
       else
         config.dynamicConfig.updateDefaultConfig(props)
     }
diff --git 
a/server-common/src/main/java/org/apache/kafka/security/PasswordEncoder.java 
b/server-common/src/main/java/org/apache/kafka/security/PasswordEncoder.java
deleted file mode 100644
index 64e11822e5f..00000000000
--- a/server-common/src/main/java/org/apache/kafka/security/PasswordEncoder.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.security;
-
-import org.apache.kafka.common.config.types.Password;
-
-import java.security.GeneralSecurityException;
-
-public interface PasswordEncoder {
-    /**
-     * A password encoder that does not modify the given password. This is 
used in KRaft mode only.
-     */
-    PasswordEncoder NOOP = new PasswordEncoder() {
-
-        @Override
-        public String encode(Password password) {
-            return password.value();
-        }
-
-        @Override
-        public Password decode(String encodedPassword) {
-            return new Password(encodedPassword);
-        }
-    };
-
-    String encode(Password password) throws GeneralSecurityException;
-    Password decode(String encodedPassword) throws GeneralSecurityException;
-}

Reply via email to