You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "omkreddy (via GitHub)" <gi...@apache.org> on 2023/02/08 16:28:50 UTC

[GitHub] [kafka] omkreddy commented on a diff in pull request #13114: KAFKA-14084: SCRAM support in KRaft.

omkreddy commented on code in PR #13114:
URL: https://github.com/apache/kafka/pull/13114#discussion_r1100299605


##########
core/src/main/scala/kafka/server/ControllerApis.scala:
##########
@@ -816,6 +817,17 @@ class ControllerApis(val requestChannel: RequestChannel,
       }
   }
 
+  def handleAlterUserScramCredentials(request: RequestChannel.Request): CompletableFuture[Unit] = {
+    val alterRequest = request.body[AlterUserScramCredentialsRequest]
+    val context = new ControllerRequestContext(request.context.header.data, request.context.principal,

Review Comment:
   We need to authorisation `authHelper.authorizeClusterOperation(request, ALTER)` check here.
   
   Similar check in KafkaAPI:
   [handleAlterUserScramCredentialsRequest](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaApis.scala#L3326)



##########
core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala:
##########
@@ -85,15 +113,17 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
           .setUpsertions(util.Arrays.asList(upsertion1, upsertion2))).build(),
     )
     requests.foreach(request => {
-      val response = sendAlterUserScramCredentialsRequest(request)
+      val response = sendAlterUserScramCredentialsRequest(request, adminSocketServer)
       val results = response.data.results
       assertEquals(2, results.size)
       checkAllErrorsAlteringCredentials(results, Errors.DUPLICATE_RESOURCE, "when altering the same credential twice in a single request")
     })
   }
 
-  @Test
-  def testAlterEmptyUser(): Unit = {
+  @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
+  @ValueSource(strings = Array("kraft", "zk"))
+  def testAlterEmptyUser(quorum: String): Unit = {
+      println("Starting test")

Review Comment:
   unwanted line



##########
core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala:
##########
@@ -221,6 +223,21 @@ class BrokerMetadataPublisher(
           s"quotas in ${deltaName}", t)
       }
 
+      // Apply changes to SCRAM credentials.
+      Option(delta.scramDelta()).foreach { scramDelta =>

Review Comment:
   May I know, how are applying changes on Controller Nodes: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ControllerServer.scala#L176



##########
metadata/src/main/java/org/apache/kafka/image/ScramCredentialData.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.image;
+
+import org.apache.kafka.clients.admin.ScramMechanism;
+import org.apache.kafka.common.metadata.UserScramCredentialRecord;
+import org.apache.kafka.common.security.scram.ScramCredential;
+import org.apache.kafka.common.security.scram.internals.ScramFormatter;
+import org.apache.kafka.common.utils.Bytes;
+
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import java.util.Objects;
+
+
+/**
+ * Represents the ACLs in the metadata image.
+ *
+ * This class is thread-safe.
+ */
+public final class ScramCredentialData {
+    private final byte[] salt;
+    private final byte[] saltedPassword;
+    private final int iterations;
+
+    static ScramCredentialData fromRecord(
+        UserScramCredentialRecord record
+    ) {
+        return new ScramCredentialData(
+                record.salt(),
+                record.saltedPassword(),
+                record.iterations());
+    }
+
+    public ScramCredentialData(
+        byte[] salt,
+        byte[] saltedPassword,
+        int iterations
+    ) {
+        this.salt = salt;
+        this.saltedPassword = saltedPassword;
+        this.iterations = iterations;
+    }
+
+    public byte[] salt() {
+        return salt;
+    }
+
+    public byte[] saltedPassword() {
+        return saltedPassword;
+    }
+
+    public int iterations() {
+        return iterations;
+    }
+
+    public UserScramCredentialRecord toRecord(
+        String userName,
+        ScramMechanism mechanism
+    ) {
+        return new UserScramCredentialRecord().
+                setName(userName).
+                setMechanism(mechanism.type()).
+                setSalt(salt).
+                setSaltedPassword(saltedPassword).
+                setIterations(iterations);
+    }
+
+    public ScramCredential toCredential(
+        ScramMechanism mechanism
+    ) throws GeneralSecurityException {
+        org.apache.kafka.common.security.scram.internals.ScramMechanism internalMechanism =
+                org.apache.kafka.common.security.scram.internals.ScramMechanism.forMechanismName(mechanism.mechanismName());
+        ScramFormatter formatter = new ScramFormatter(internalMechanism);
+        return new ScramCredential(salt,
+                formatter.storedKey(formatter.clientKey(saltedPassword)),
+                formatter.serverKey(saltedPassword),
+                iterations);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(salt, saltedPassword, iterations);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null) return false;
+        if (!o.getClass().equals(ScramCredentialData.class)) return false;
+        ScramCredentialData other = (ScramCredentialData) o;
+        return Arrays.equals(salt, other.salt) &&
+                Arrays.equals(saltedPassword, other.saltedPassword) &&
+                iterations == other.iterations;
+    }
+
+    @Override
+    public String toString() {
+        return "ScramCredentialData" +

Review Comment:
   I think, we should not return sensitive info like salt, saltedPassword etc..



##########
metadata/src/main/java/org/apache/kafka/controller/ScramControlManager.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.controller;
+
+import org.apache.kafka.clients.admin.ScramMechanism;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialDeletion;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialUpsertion;
+import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
+import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult;
+import org.apache.kafka.common.metadata.RemoveUserScramCredentialRecord;
+import org.apache.kafka.common.metadata.UserScramCredentialRecord;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import static org.apache.kafka.common.protocol.Errors.DUPLICATE_RESOURCE;
+import static org.apache.kafka.common.protocol.Errors.NONE;
+import static org.apache.kafka.common.protocol.Errors.RESOURCE_NOT_FOUND;
+import static org.apache.kafka.common.protocol.Errors.UNACCEPTABLE_CREDENTIAL;
+import static org.apache.kafka.common.protocol.Errors.UNSUPPORTED_SASL_MECHANISM;
+
+
+/**
+ * Manages SCRAM credentials.
+ */
+public class ScramControlManager {
+    static final int MAX_ITERATIONS = 16384;
+
+    static class Builder {
+        private LogContext logContext = null;
+        private SnapshotRegistry snapshotRegistry = null;
+
+        Builder setLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        Builder setSnapshotRegistry(SnapshotRegistry snapshotRegistry) {
+            this.snapshotRegistry = snapshotRegistry;
+            return this;
+        }
+
+        ScramControlManager build() {
+            if (logContext == null) logContext = new LogContext();
+            if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
+            return new ScramControlManager(logContext,
+                snapshotRegistry);
+        }
+    }
+
+    static class ScramCredentialKey {
+        private final String username;
+        private final ScramMechanism mechanism;
+
+        ScramCredentialKey(String username, ScramMechanism mechanism) {
+            this.username = username;
+            this.mechanism = mechanism;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(username, mechanism);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o == null) return false;
+            if (!(o.getClass() == this.getClass())) return false;
+            ScramCredentialKey other = (ScramCredentialKey) o;
+            return username.equals(other.username) &&
+                mechanism.equals(other.mechanism);
+        }
+
+        @Override
+        public String toString() {
+            return "ScramCredentialKey" +
+                "(username=" + username +
+                ", mechanism=" + mechanism +
+                ")";
+        }
+    }
+
+    static class ScramCredentialValue {
+        private final byte[] salt;
+        private final byte[] saltedPassword;
+        private final int iterations;
+
+        ScramCredentialValue(
+            byte[] salt,
+            byte[] saltedPassword,
+            int iterations
+        ) {
+            this.salt = salt;
+            this.saltedPassword = saltedPassword;
+            this.iterations = iterations;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(salt, saltedPassword, iterations);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o == null) return false;
+            if (!(o.getClass() == this.getClass())) return false;
+            ScramCredentialValue other = (ScramCredentialValue) o;
+            return Arrays.equals(salt, other.salt) &&
+                Arrays.equals(saltedPassword, other.saltedPassword) &&
+                iterations == other.iterations;
+        }
+
+        @Override
+        public String toString() {
+            return "ScramCredentialValue" +
+                "(salt=" + Base64.getEncoder().encodeToString(salt) +
+                ", saltedPassword=" + Base64.getEncoder().encodeToString(saltedPassword) +
+                ", iterations=" + iterations +
+                ")";
+        }
+    }
+
+    private final Logger log;
+    private final TimelineHashMap<ScramCredentialKey, ScramCredentialValue> credentials;
+
+    private ScramControlManager(
+        LogContext logContext,
+        SnapshotRegistry snapshotRegistry
+    ) {
+        this.log = logContext.logger(ScramControlManager.class);
+        this.credentials = new TimelineHashMap<>(snapshotRegistry, 0);
+    }
+
+    public ControllerResult<AlterUserScramCredentialsResponseData> alterCredentials(
+        AlterUserScramCredentialsRequestData request
+    ) {
+        Map<String, ScramCredentialDeletion> userToDeletion = new HashMap<>();
+        Map<String, ScramCredentialUpsertion> userToUpsert = new HashMap<>();
+        Map<String, ApiError> userToError = new HashMap<>();
+
+        for (ScramCredentialDeletion deletion : request.deletions()) {
+            if (!userToError.containsKey(deletion.name())) {
+                if (userToDeletion.remove(deletion.name()) != null) {
+                    userToError.put(deletion.name(), new ApiError(DUPLICATE_RESOURCE,
+                        "A user credential cannot be altered twice in the same request"));
+                } else {
+                    ApiError error = validateDeletion(deletion);
+                    if (error.isFailure()) {
+                        userToError.put(deletion.name(), error);
+                    } else {
+                        userToDeletion.put(deletion.name(), deletion);
+                    }
+                }
+            }
+        }
+        for (ScramCredentialUpsertion upsertion : request.upsertions()) {
+            if (!userToError.containsKey(upsertion.name())) {
+                if (userToDeletion.remove(upsertion.name()) != null ||
+                        userToUpsert.remove(upsertion.name()) != null) {
+                    userToError.put(upsertion.name(), new ApiError(DUPLICATE_RESOURCE,
+                        "A user credential cannot be altered twice in the same request"));
+                } else {
+                    ApiError error = validateUpsertion(upsertion);
+                    if (error.isFailure()) {
+                        userToError.put(upsertion.name(), error);
+                    } else {
+                        userToUpsert.put(upsertion.name(), upsertion);
+                    }
+                }
+            }
+        }
+        AlterUserScramCredentialsResponseData response = new AlterUserScramCredentialsResponseData();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        for (ScramCredentialDeletion deletion : userToDeletion.values()) {
+            response.results().add(new AlterUserScramCredentialsResult().
+                setUser(deletion.name()).
+                setErrorCode(NONE.code()).
+                setErrorMessage(null));
+            records.add(new ApiMessageAndVersion(new RemoveUserScramCredentialRecord().
+                setName(deletion.name()).
+                setMechanism(deletion.mechanism()), (short) 0));
+        }
+        for (ScramCredentialUpsertion upsertion : userToUpsert.values()) {
+            response.results().add(new AlterUserScramCredentialsResult().
+                setUser(upsertion.name()).
+                setErrorCode(NONE.code()).
+                setErrorMessage(null));
+            records.add(new ApiMessageAndVersion(new UserScramCredentialRecord().
+                setName(upsertion.name()).
+                setMechanism(upsertion.mechanism()).
+                setSalt(upsertion.salt()).
+                setSaltedPassword(upsertion.saltedPassword()).
+                setIterations(upsertion.iterations()), (short) 0));
+        }
+        for (Entry<String, ApiError> entry : userToError.entrySet()) {
+            response.results().add(new AlterUserScramCredentialsResult().
+                setUser(entry.getKey()).
+                setErrorCode(entry.getValue().error().code()).
+                setErrorMessage(entry.getValue().message()));
+        }
+        return ControllerResult.atomicOf(records, response);
+    }
+
+    static ApiError validateUpsertion(ScramCredentialUpsertion upsertion) {
+        ScramMechanism mechanism = ScramMechanism.fromType(upsertion.mechanism());
+        ApiError error = validateScramUsernameAndMechanism(upsertion.name(), mechanism);
+        if (error.isFailure()) return error;
+        org.apache.kafka.common.security.scram.internals.ScramMechanism internalMechanism =
+            org.apache.kafka.common.security.scram.internals.ScramMechanism.forMechanismName(mechanism.mechanismName());
+        if (upsertion.iterations() < internalMechanism.minIterations()) {
+            return new ApiError(UNACCEPTABLE_CREDENTIAL, "Too few iterations");
+        } else if (upsertion.iterations() > MAX_ITERATIONS) {
+            return new ApiError(UNACCEPTABLE_CREDENTIAL, "Too many iterations");
+        }
+        return ApiError.NONE;
+    }
+
+    ApiError validateDeletion(ScramCredentialDeletion deletion) {
+        ApiError error = validateScramUsernameAndMechanism(deletion.name(),
+            ScramMechanism.fromType(deletion.mechanism()));
+        if (error.isFailure()) return error;
+        ScramCredentialKey key = new ScramCredentialKey(deletion.name(),
+            ScramMechanism.fromType(deletion.mechanism()));
+        if (!credentials.containsKey(key)) {
+            return new ApiError(RESOURCE_NOT_FOUND,
+                "Attempt to delete a user credential that does not exist");
+        }
+        return ApiError.NONE;
+    }
+
+    static ApiError validateScramUsernameAndMechanism(
+        String username,
+        ScramMechanism mechanism
+    ) {
+        if (username.isEmpty()) {
+            return new ApiError(UNACCEPTABLE_CREDENTIAL, "Username must not be empty");
+        }
+        if (mechanism == ScramMechanism.UNKNOWN) {
+            return new ApiError(UNSUPPORTED_SASL_MECHANISM, "Unknown SCRAM mechanism");
+        }
+        return ApiError.NONE;
+    }
+
+    public void replay(RemoveUserScramCredentialRecord record) {
+        ScramCredentialKey key = new ScramCredentialKey(record.name(),
+            ScramMechanism.fromType(record.mechanism()));
+        if (credentials.remove(key) == null) {
+            throw new RuntimeException("Unable to find credential to delete: " + key);
+        }
+        log.info("Removed SCRAM credential for {} with mechanism {}.",
+            key.username, key.mechanism);
+    }
+
+    public void replay(UserScramCredentialRecord record) {
+        ScramCredentialKey key = new ScramCredentialKey(record.name(),
+            ScramMechanism.fromType(record.mechanism()));
+        ScramCredentialValue value = new ScramCredentialValue(record.salt(),
+            record.saltedPassword(),
+            record.iterations());
+        if (credentials.put(key, value) == null) {
+            log.info("Created new SCRAM credential for {} with mechanism {}.",
+                key.username, key.mechanism);
+        } else {
+            log.info("Modified SCRAM credential for {} with mechanism {}.",
+                key.username, key.mechanism);
+        }
+    }
+
+    ApiMessageAndVersion toRecord(ScramCredentialKey key, ScramCredentialValue value) {

Review Comment:
   unused method



##########
metadata/src/main/java/org/apache/kafka/controller/ScramControlManager.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.controller;
+
+import org.apache.kafka.clients.admin.ScramMechanism;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialDeletion;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialUpsertion;
+import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
+import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult;
+import org.apache.kafka.common.metadata.RemoveUserScramCredentialRecord;
+import org.apache.kafka.common.metadata.UserScramCredentialRecord;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import static org.apache.kafka.common.protocol.Errors.DUPLICATE_RESOURCE;
+import static org.apache.kafka.common.protocol.Errors.NONE;
+import static org.apache.kafka.common.protocol.Errors.RESOURCE_NOT_FOUND;
+import static org.apache.kafka.common.protocol.Errors.UNACCEPTABLE_CREDENTIAL;
+import static org.apache.kafka.common.protocol.Errors.UNSUPPORTED_SASL_MECHANISM;
+
+
+/**
+ * Manages SCRAM credentials.
+ */
+public class ScramControlManager {
+    static final int MAX_ITERATIONS = 16384;
+
+    static class Builder {
+        private LogContext logContext = null;
+        private SnapshotRegistry snapshotRegistry = null;
+
+        Builder setLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        Builder setSnapshotRegistry(SnapshotRegistry snapshotRegistry) {
+            this.snapshotRegistry = snapshotRegistry;
+            return this;
+        }
+
+        ScramControlManager build() {
+            if (logContext == null) logContext = new LogContext();
+            if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
+            return new ScramControlManager(logContext,
+                snapshotRegistry);
+        }
+    }
+
+    static class ScramCredentialKey {
+        private final String username;
+        private final ScramMechanism mechanism;
+
+        ScramCredentialKey(String username, ScramMechanism mechanism) {
+            this.username = username;
+            this.mechanism = mechanism;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(username, mechanism);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o == null) return false;
+            if (!(o.getClass() == this.getClass())) return false;
+            ScramCredentialKey other = (ScramCredentialKey) o;
+            return username.equals(other.username) &&
+                mechanism.equals(other.mechanism);
+        }
+
+        @Override
+        public String toString() {
+            return "ScramCredentialKey" +
+                "(username=" + username +
+                ", mechanism=" + mechanism +
+                ")";
+        }
+    }
+
+    static class ScramCredentialValue {
+        private final byte[] salt;
+        private final byte[] saltedPassword;
+        private final int iterations;
+
+        ScramCredentialValue(
+            byte[] salt,
+            byte[] saltedPassword,
+            int iterations
+        ) {
+            this.salt = salt;
+            this.saltedPassword = saltedPassword;
+            this.iterations = iterations;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(salt, saltedPassword, iterations);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o == null) return false;
+            if (!(o.getClass() == this.getClass())) return false;
+            ScramCredentialValue other = (ScramCredentialValue) o;
+            return Arrays.equals(salt, other.salt) &&
+                Arrays.equals(saltedPassword, other.saltedPassword) &&
+                iterations == other.iterations;
+        }
+
+        @Override
+        public String toString() {
+            return "ScramCredentialValue" +
+                "(salt=" + Base64.getEncoder().encodeToString(salt) +
+                ", saltedPassword=" + Base64.getEncoder().encodeToString(saltedPassword) +
+                ", iterations=" + iterations +
+                ")";
+        }
+    }
+
+    private final Logger log;
+    private final TimelineHashMap<ScramCredentialKey, ScramCredentialValue> credentials;
+
+    private ScramControlManager(
+        LogContext logContext,
+        SnapshotRegistry snapshotRegistry
+    ) {
+        this.log = logContext.logger(ScramControlManager.class);
+        this.credentials = new TimelineHashMap<>(snapshotRegistry, 0);
+    }
+
+    public ControllerResult<AlterUserScramCredentialsResponseData> alterCredentials(

Review Comment:
   This method is  important for applying changes. Can we add few unit tests scenarios to verify this method.



##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -53,7 +53,8 @@ import org.apache.kafka.common.requests._
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.utils.{LogContext, Time, Utils}
 import org.apache.kafka.common.{IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid}
-import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, FeaturesImage, MetadataImage, MetadataProvenance, ProducerIdsImage, TopicsDelta, TopicsImage}
+import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, FeaturesImage,

Review Comment:
   maybe we can use `import org.apache.kafka.image._`



##########
core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala:
##########
@@ -42,10 +46,18 @@ import scala.jdk.CollectionConverters._
  * Also tests the Alter and Describe APIs for the case where credentials are successfully altered/described.
  */
 class AlterUserScramCredentialsRequestTest extends BaseRequestTest {

Review Comment:
   we should also support Describe API and update `testAlterAndDescribe` method



##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -1529,6 +1539,11 @@ private void resetToEmptyState() {
      */
     private final ReplicationControlManager replicationControl;
 
+    /**
+     * Manages SCRAM credentials, if there are any.
+     */
+    private final ScramControlManager scramControl;

Review Comment:
   `scramControl` => `scramControlManager`



##########
metadata/src/main/java/org/apache/kafka/image/ScramImage.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.image;
+
+import org.apache.kafka.image.writer.ImageWriter;
+import org.apache.kafka.image.writer.ImageWriterOptions;
+import org.apache.kafka.clients.admin.ScramMechanism;
+
+// import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+// import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+
+/**
+ * Represents the SCRAM credentials in the metadata image.
+ *
+ * This class is thread-safe.
+ */
+public final class ScramImage {
+    public static final ScramImage EMPTY = new ScramImage(Collections.emptyMap());
+
+    private final Map<ScramMechanism, Map<String, ScramCredentialData>> mechanisms;
+
+    public ScramImage(Map<ScramMechanism, Map<String, ScramCredentialData>> mechanisms) {
+        this.mechanisms = Collections.unmodifiableMap(mechanisms);
+    }
+
+    public void write(ImageWriter writer, ImageWriterOptions options) {

Review Comment:
   ImageWriterOptions is unused.



##########
metadata/src/main/java/org/apache/kafka/image/ScramImage.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.image;
+
+import org.apache.kafka.image.writer.ImageWriter;
+import org.apache.kafka.image.writer.ImageWriterOptions;
+import org.apache.kafka.clients.admin.ScramMechanism;
+
+// import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+// import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+
+/**
+ * Represents the SCRAM credentials in the metadata image.
+ *
+ * This class is thread-safe.
+ */
+public final class ScramImage {
+    public static final ScramImage EMPTY = new ScramImage(Collections.emptyMap());
+
+    private final Map<ScramMechanism, Map<String, ScramCredentialData>> mechanisms;
+
+    public ScramImage(Map<ScramMechanism, Map<String, ScramCredentialData>> mechanisms) {
+        this.mechanisms = Collections.unmodifiableMap(mechanisms);
+    }
+
+    public void write(ImageWriter writer, ImageWriterOptions options) {
+        for (Entry<ScramMechanism, Map<String, ScramCredentialData>> mechanismEntry : mechanisms.entrySet()) {
+            for (Entry<String, ScramCredentialData> userEntry : mechanismEntry.getValue().entrySet()) {
+                writer.write(0, userEntry.getValue().toRecord(userEntry.getKey(), mechanismEntry.getKey()));
+            }
+        }
+    }
+
+    public Map<ScramMechanism, Map<String, ScramCredentialData>> mechanisms() {
+        return mechanisms;
+    }
+
+    public boolean isEmpty() {
+        return mechanisms.isEmpty();
+    }
+
+    @Override
+    public int hashCode() {
+        return mechanisms.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null) return false;
+        if (!o.getClass().equals(ScramImage.class)) return false;
+        ScramImage other = (ScramImage) o;
+        return mechanisms.equals(other.mechanisms);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        builder.append("ScramImage(");
+        List<ScramMechanism> sortedMechanisms = mechanisms.keySet().stream().sorted().collect(Collectors.toList());
+        String preMechanismComma = "";
+        for (ScramMechanism mechanism : sortedMechanisms) {
+            builder.append(preMechanismComma).append(mechanism).append(": {");
+            Map<String, ScramCredentialData> userMap = mechanisms.get(mechanism);
+            List<String> sortedUserNames = userMap.keySet().stream().sorted().collect(Collectors.toList());
+            String preUserNameComma = "";
+            for (String userName : sortedUserNames) {
+                builder.append(preUserNameComma).append(userName).append("=").append(userMap.get(userName));

Review Comment:
   we should not print  sensitive info like `userMap.get(`



##########
core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala:
##########
@@ -221,6 +223,21 @@ class BrokerMetadataPublisher(
           s"quotas in ${deltaName}", t)
       }
 
+      // Apply changes to SCRAM credentials.
+      Option(delta.scramDelta()).foreach { scramDelta =>

Review Comment:
   I think we need to update `ScramControlManager.reply` method to call `credentialProvider. updateCredential(), removeCredentials()` methods



##########
metadata/src/main/java/org/apache/kafka/controller/ScramControlManager.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.controller;
+
+import org.apache.kafka.clients.admin.ScramMechanism;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialDeletion;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialUpsertion;
+import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
+import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult;
+import org.apache.kafka.common.metadata.RemoveUserScramCredentialRecord;
+import org.apache.kafka.common.metadata.UserScramCredentialRecord;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import static org.apache.kafka.common.protocol.Errors.DUPLICATE_RESOURCE;
+import static org.apache.kafka.common.protocol.Errors.NONE;
+import static org.apache.kafka.common.protocol.Errors.RESOURCE_NOT_FOUND;
+import static org.apache.kafka.common.protocol.Errors.UNACCEPTABLE_CREDENTIAL;
+import static org.apache.kafka.common.protocol.Errors.UNSUPPORTED_SASL_MECHANISM;
+
+
+/**
+ * Manages SCRAM credentials.
+ */
+public class ScramControlManager {
+    static final int MAX_ITERATIONS = 16384;
+
+    static class Builder {
+        private LogContext logContext = null;
+        private SnapshotRegistry snapshotRegistry = null;
+
+        Builder setLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        Builder setSnapshotRegistry(SnapshotRegistry snapshotRegistry) {
+            this.snapshotRegistry = snapshotRegistry;
+            return this;
+        }
+
+        ScramControlManager build() {
+            if (logContext == null) logContext = new LogContext();
+            if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
+            return new ScramControlManager(logContext,
+                snapshotRegistry);
+        }
+    }
+
+    static class ScramCredentialKey {
+        private final String username;
+        private final ScramMechanism mechanism;
+
+        ScramCredentialKey(String username, ScramMechanism mechanism) {
+            this.username = username;
+            this.mechanism = mechanism;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(username, mechanism);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o == null) return false;
+            if (!(o.getClass() == this.getClass())) return false;
+            ScramCredentialKey other = (ScramCredentialKey) o;
+            return username.equals(other.username) &&
+                mechanism.equals(other.mechanism);
+        }
+
+        @Override
+        public String toString() {
+            return "ScramCredentialKey" +
+                "(username=" + username +
+                ", mechanism=" + mechanism +
+                ")";
+        }
+    }
+
+    static class ScramCredentialValue {
+        private final byte[] salt;
+        private final byte[] saltedPassword;
+        private final int iterations;
+
+        ScramCredentialValue(
+            byte[] salt,
+            byte[] saltedPassword,
+            int iterations
+        ) {
+            this.salt = salt;
+            this.saltedPassword = saltedPassword;
+            this.iterations = iterations;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(salt, saltedPassword, iterations);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o == null) return false;
+            if (!(o.getClass() == this.getClass())) return false;
+            ScramCredentialValue other = (ScramCredentialValue) o;
+            return Arrays.equals(salt, other.salt) &&
+                Arrays.equals(saltedPassword, other.saltedPassword) &&
+                iterations == other.iterations;
+        }
+
+        @Override
+        public String toString() {
+            return "ScramCredentialValue" +

Review Comment:
   We should return sensitive data



-- 
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-unsubscribe@kafka.apache.org

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