You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/07/16 22:50:27 UTC

[GitHub] [kafka] rondagostino opened a new pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

rondagostino opened a new pull request #9032:
URL: https://github.com/apache/kafka/pull/9032


   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472574569



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       If we do decide to go with 2 separate APIs, then I might be concerned about using `list()` vs `describe()` if they return the same set of information (i.e. mechanism and iterations).  Perhaps using two separate names gives us room to expand `describe()` to return more information later on, though.  But if not, and they will always return the same information, then maybe `describeAll()` and `describe()` (or `listAll()` and `list()`) might be better?




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r478653535



##########
File path: tests/kafkatest/services/security/security_config.py
##########
@@ -350,6 +362,14 @@ def kafka_opts(self):
         else:
             return ""
 
+    def export_kafka_opts_for_admin_client_as_broker(self):

Review comment:
       Yes, created https://issues.apache.org/jira/browse/KAFKA-10443 to track this.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479553827



##########
File path: core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala
##########
@@ -42,7 +42,18 @@ class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTes
   override def setUp(): Unit = {
     super.setUp()
     // Create client credentials after starting brokers so that dynamic credential creation is also tested
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
+    createScramCredentialWithScramAdminClient(JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
+    createScramCredentialWithScramAdminClient(JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
+  }
+
+  private def createScramCredentialWithScramAdminClient(user: String, password: String) = {

Review comment:
       So the reason for not using the `SaslSertup#createScramCredentials` method here is because we want the admin client itself to be authenticated with SCRAM?




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

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



[GitHub] [kafka] cmccabe commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-685130207


   ok to test


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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r474950597



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<UserScramCredentialsDescriptionResult>> future;
+
+    /**
+     *
+     * @param future the required future representing the result of the call
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<UserScramCredentialsDescriptionResult>> future) {
+        this.future = Objects.requireNonNull(future);
+    }
+
+    /**
+     *
+     * @return the future representing the result of the call
+     */
+    public KafkaFuture<List<UserScramCredentialsDescriptionResult>> future() {

Review comment:
       Sorry, my mistake.  This now reflects what we discussed.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465374092



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));

Review comment:
       We should have a defined ApiException and Errors entry for errors like this.  We could maybe use InvalidConfigurationException / INVALID_CONFIG 




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

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



[GitHub] [kafka] rondagostino commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-685957655


   System test run was successful except for 2 known flaky tests.  https://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2020-09-02--001.1599074927--rondagostino--KAFKA-10259--b7d1fee5a/report.txt
   
   ```
   passed:           512
   failed:           2
   
   test_id:    kafkatest.tests.connect.connect_distributed_test.ConnectDistributedTest.test_bounce.connect_protocol=compatible.clean=True
   status:     FAIL
   
   test_id:    kafkatest.tests.core.replication_test.ReplicationTest.test_replication_with_broker_failure.security_protocol=SASL_SSL.failure_mode=clean_shutdown.broker_type=leader
   status:     FAIL
   ```


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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r474236159



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       Ok, we have two levels of Futures now:
   
   1. `DescribeUserScramCredentialsResult` has a `KafkaFuture<List<UserScramCredentialsDescriptionResult>>`
   2. `UserScramCredentialsDescriptionResult` has a user name and a `KafkaFuture<UserScramCredentialsDescription>`
   
   I think this is where we want to end up.  Let me know if you agree.  I also added some logic in `AlterUserScramCredentialsRequestTest` to confirm the behavior of the per-user futures and associated errors.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479548642



##########
File path: core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala
##########
@@ -166,14 +168,11 @@ class ClientQuotasRequestTest extends BaseRequestTest {
 
   @Test
   def testClientQuotasForScramUsers(): Unit = {
-    val entityType = ConfigType.User
     val userName = "user"
 
-    val mechanism = ScramMechanism.SCRAM_SHA_256
-    val credential = new ScramFormatter(mechanism).generateCredential("password", 4096)
-    val configs = adminZkClient.fetchEntityConfig(entityType, userName)
-    configs.setProperty(mechanism.mechanismName, ScramCredentialUtils.credentialToString(credential))
-    adminZkClient.changeConfigs(entityType, userName, configs)
+    val results = createAdminClient().alterUserScramCredentials(util.Arrays.asList(

Review comment:
       You need to close the admin client after creating it.  I don't think Scala has try-with-resources, so you can just use a `try`... `finally` block.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468881832



##########
File path: core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 kafka.server
+
+import java.util
+import java.util.Properties
+
+import kafka.network.SocketServer
+import kafka.security.authorizer.AclAuthorizer
+import org.apache.kafka.common.acl.AclOperation
+import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData
+import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
+import org.apache.kafka.common.resource.ResourceType
+import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder}
+import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
+import org.junit.Assert._
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Test DescribeUserScramCredentialsRequest/Response API for the cases where no credentials exist
+ * or failure is expected due to lack of authorization, sending the request to a non-controller broker, or some other issue.
+ * Testing the API for the case where there are actually credentials to describe is performed elsewhere.
+ */
+class DescribeUserScramCredentialsRequestTest extends BaseRequestTest {
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.ControlledShutdownEnableProp, "false")
+    properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[DescribeCredentialsTest.TestAuthorizer].getName)
+    properties.put(KafkaConfig.PrincipalBuilderClassProp, classOf[DescribeCredentialsTest.TestPrincipalBuilder].getName)
+  }
+
+  @Before
+  override def setUp(): Unit = {
+    DescribeCredentialsTest.principal = KafkaPrincipal.ANONYMOUS // default is to be authorized
+    super.setUp()
+  }
+
+  @Test
+  def testDescribeNothing(): Unit = {
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response = sendDescribeUserScramCredentialsRequest(request)
+
+    val error = response.data.error
+    assertEquals("Expected no error when routed correctly", Errors.NONE.code, error)
+    assertEquals("Expected no credentials", 0, response.data.userScramCredentials.size)
+  }
+
+  @Test
+  def testDescribeNotController(): Unit = {
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response = sendDescribeUserScramCredentialsRequest(request, notControllerSocketServer)
+
+    val error = response.data.error
+    assertEquals("Expected controller error when routed incorrectly", Errors.NOT_CONTROLLER.code, error)
+  }
+
+  @Test
+  def testDescribeNotAuthorized(): Unit = {
+    DescribeCredentialsTest.principal = DescribeCredentialsTest.UnauthorizedPrincipal
+
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response = sendDescribeUserScramCredentialsRequest(request)
+
+    val error = response.data.error
+    assertEquals("Expected not authorized error", Errors.CLUSTER_AUTHORIZATION_FAILED.code, error)
+  }
+
+  @Test
+  def testDescribeSameUserTwice(): Unit = {
+    val user = new UserName().setName("user1")
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData().setUsers(List(user, user).asJava)).build()
+    val response = sendDescribeUserScramCredentialsRequest(request)
+
+    val error = response.data.error
+    assertEquals("Expected invalid request error", Errors.INVALID_REQUEST.code, error)
+  }
+
+
+  private def sendDescribeUserScramCredentialsRequest(request: DescribeUserScramCredentialsRequest, socketServer: SocketServer = controllerSocketServer): DescribeUserScramCredentialsResponse = {
+    connectAndReceive[DescribeUserScramCredentialsResponse](request, destination = socketServer)
+  }
+}
+
+object DescribeCredentialsTest {
+  val UnauthorizedPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "Unauthorized")

Review comment:
       It's kind of a code smell to be mutating static data during a JUnit test.  There are ways to run JUnit tests in parallel (although I think our current setup only runs at the granularity of test classes, not test methods?)  At the very least we'd have to document that this is not thread-safe and that it will prevent us from parellelizing this test class in the future.
   
   Rather than doing this, how about having two separate test authorizers, and choosing the one you want for each individual JUnit test method?
   
   In order to fit this into our bass-akwards inheritance based test design ( :( ) you could check the name of the test in `brokerPropertyOverrides` using one of these methods: https://stackoverflow.com/questions/473401/get-name-of-currently-executing-test-in-junit-4




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465751005



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new IllegalArgumentException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite aa exception from a previous upsertion, but we don't really care
+                        // since we just needs to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);

Review comment:
       I wonder if `InvalidRequestException` is indeed the right one.  On the broker side, in `kafka.server.AdminManager`, the code looks like this:
   
   ```
       invalidUsers.foreach(user => retval.results.add(new AlterUserScramCredentialsResult().setUser(user)
         .setErrorCode(Errors.INVALID_REQUEST.code).setErrorMessage("Unknown SCRAM mechanism or too few iterations")))
   ```
   




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480301973



##########
File path: core/src/test/scala/unit/kafka/admin/UserScramCredentialsCommandTest.scala
##########
@@ -0,0 +1,135 @@
+/**
+ * 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 kafka.admin
+
+import java.io.{ByteArrayOutputStream, PrintStream}
+import java.nio.charset.StandardCharsets
+
+import kafka.server.BaseRequestTest
+import kafka.utils.Exit
+import org.junit.Assert._
+import org.junit.Test
+
+class UserScramCredentialsCommandTest extends BaseRequestTest {
+  override def brokerCount = 1
+  var exitStatus: Option[Int] = None
+  var exitMessage: Option[String] = None
+
+  case class ConfigCommandResult(stdout: String, exitStatus: Option[Int] = None)
+
+  private def runConfigCommandViaBroker(args: Array[String]) : ConfigCommandResult = {
+    val byteArrayOutputStream = new ByteArrayOutputStream()
+    val utf8 = StandardCharsets.UTF_8.name
+    val printStream = new PrintStream(byteArrayOutputStream, true, utf8)
+    var exitStatus: Option[Int] = None
+    Exit.setExitProcedure { (status, _) =>
+      exitStatus = Some(status)
+      throw new RuntimeException
+    }
+    try {
+      Console.withOut(printStream) {
+        ConfigCommand.main(Array("--bootstrap-server", brokerList) ++ args)
+      }
+      ConfigCommandResult(byteArrayOutputStream.toString(utf8))
+    } catch {
+      case e: Exception => {

Review comment:
       Logging it at debug level doesn't hurt, so I added it.




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

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



[GitHub] [kafka] rajinisivaram commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468632388



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.fromType(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new InvalidRequestException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new InvalidRequestException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite an exception from a previous upsertion, but we don't really care
+                        // since we just need to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);
+                    }
+                });
+        // fail any users immediately that have an illegal alteration as identified above
+        userIllegalAlterationExceptions.entrySet().stream().forEach(entry -> {
+            futures.get(entry.getKey()).completeExceptionally(entry.getValue());
+        });
+
+        // submit alterations for users that do not have an illegal upsertion as identified above
+        Call call = new Call("alterUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public AlterUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new AlterUserScramCredentialsRequest.Builder(
+                        new AlterUserScramCredentialsRequestData().setUpsertions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialUpsertion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(a -> userInsertions.get(a.getUser()).get(((UserScramCredentialUpsertion) a).getInfo().getMechanism()))
+                                .collect(Collectors.toList()))
+                        .setDeletions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialDeletion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(d ->
+                                getScramCredentialDeletion((UserScramCredentialDeletion) d)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                AlterUserScramCredentialsResponse response = (AlterUserScramCredentialsResponse) abstractResponse;
+                // Check for controller change
+                for (Errors error : response.errorCounts().keySet()) {
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                    }
+                }
+                response.data().results().forEach(result -> {
+                    KafkaFutureImpl<Void> future = futures.get(result.user());
+                    if (future == null) {
+                        log.warn("Server response mentioned unknown user {}", result.user());
+                    } else {
+                        Errors error = Errors.forCode(result.errorCode());
+                        if (error != Errors.NONE) {
+                            future.completeExceptionally(error.exception(result.errorMessage()));
+                        } else {
+                            future.complete(null);
+                        }
+                    }
+                });
+                completeUnrealizedFutures(
+                    futures.entrySet().stream(),
+                    user -> "The broker response did not contain a result for user " + user);
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(futures.values(), throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new AlterUserScramCredentialsResult(new HashMap<>(futures));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialUpsertion getScramCredentialUpsertion(UserScramCredentialUpsertion u) throws InvalidKeyException, NoSuchAlgorithmException {
+        AlterUserScramCredentialsRequestData.ScramCredentialUpsertion retval = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion();
+        return retval.setName(u.getUser())
+                .setMechanism(u.getInfo().getMechanism().getType())
+                .setIterations(u.getInfo().getIterations())
+                .setSalt(u.getSalt())
+                .setSaltedPassword(getSaltedPasword(u.getInfo().getMechanism(), u.getPassword(), u.getSalt(), u.getInfo().getIterations()));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialDeletion getScramCredentialDeletion(UserScramCredentialDeletion d) {
+        return new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(d.getUser()).setMechanism(d.getMechanism().getType());
+    }
+
+    private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte[] password, byte[] salt, int iterations) throws NoSuchAlgorithmException, InvalidKeyException {
+        return new ScramFormatter(org.apache.kafka.common.security.scram.internals.ScramMechanism.forMechanismName(publicScramMechanism.getMechanismName()))
+                .hi(password, salt, iterations);

Review comment:
       Agree that we should just get rid of -1.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479554324



##########
File path: core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala
##########
@@ -248,4 +250,25 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with
     producerConfig.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
     createProducer()
   }
+
+  private def createScramAdminClient(user: String, password: String): Admin = {

Review comment:
       It feels like we are accumulating a lot of these "create an admin client, but with SCRAM" functions.  Since all these tests ultimately subclass SaslSetup, can't we have a common function there?




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

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



[GitHub] [kafka] rondagostino commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-672110976


   @cmccabe and @rajinisivaram Thanks for the reviews.  I think this is where we stand:
   
   1. I removed `-1` as a valid `iterations` value in the code.  I will need to announce this in the DISCUSS email thread and adjust the KIP.  I will do that once we have the full set of KIP changes based on all these points.
   2. We decided that we will not allow users who authenticated using delegation tokens to alter user SRAM credentials.  I will add a mention of this to the KIP once we have the full set of KIP changes based on all these points.
   2. The KIP says that Describe `will be sent to the controller, and will return NOT_CONTROLLER if the receiving broker is not the controller.`. This constraint is perhaps not necessary for Describe requests.  If we are all in agreement then I can change the code, and I will adjust the KIP once we have the full set of KIP changes based on all these points.
   3. We have to decide what to do if a client asks to Describe specific users that don't exist.  The KIP does not explicitly say what to do in this situation, but the Describe Response only has a top-level error; unlike the Alterations response, the Describe response doesn't provide an error per user.  So the entire Describe request currently has to succeed or fail as a whole.  Therefore the code currently just silently ignores users that don't exist and only describes users that do exist (i.e. if you request to describe `user1`, `user2`, and `user3`, but `user2` doesn't exist (or it does exist but doesn't have any SCRAM credentials), the response indicates success and describes just `user1` and `user3`.  We need to decide if the current behavior is acceptable, and if it is acceptable, I would add a line to the KIP making this behavior explicit (since I just implied it based on the single error design).  If this behavior is not acceptable, then I think we would need to adjust the fo
 rmat of the response, the code, and the KIP.  Thoughts?
   4. I updated existing both the `integration tests` and `system tests` to use the admin client when creating User SCRAM credentials after the brokers start instead of going directly to ZooKeeper; ZooKeeper is still used for bootstrapping broker credentials prior to starting Kafka, of course.  These changes haven't been reviewed yet, I don't think, and we should probably kick off test runs to exercise everything (tests pass locally for me).
   


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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469680702



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       Now that I'm working on this, I discovered that there is no other API that can describe or list everything that works this way.  Everything that can describe or list everything returns a single future.  Every describe or list API that returns a map of keys to futures requires a non-empty list of keys to describe or list.  For example:
   
   1. `listTopics()` lists all topics and returns a single `Future`; the `describeTopics()` API returns a map of names to futures but requires a non-empty list of topics to describe.
   2. `describeConfigs()` returns a map of resources to futures but requires a non-empty list of resources to describe.
   3. `describeLogDirs()` returns a map of broker IDs to futures but requires a non-empty list of brokers to describe.
   4. `describeReplicaLogDirs()` returns a map of replicas to futures but requires a non-empty list of replicas to describe.
   5. `describeConsumerGroups()` returns a map of consumer groups to futures but requires a non-empty list of consumer groups to describe.
   6. `listPartitionReassignments()` allows listing all or a subset of reassignments and returns a single future.
   7. `listOffsets()` returns a map of topic-partitions to futures but requires a non-empty list of topic-partitions to describe.
   8. `describeClientQuotas()` allows listing all or a subset of quotas and returns a single future.
   
   I think if we made this change here we would be off the beaten path.  That's not necessarily bad, but what tipped me off to this was the fact that when we list everything we have to create a future for every user that gets returned, and we don't know that list of users when we make the request, so there's really no way to implement it.
   
   We could create two separate APIs: one for describing some explicit, non-empty list of users, which would return a map of users to futures, and another one that describes everything, which returns a single future.  `listTopics()` vs `describeTopics()` works this way, for example, though the information returned in the two is very different: when listing you just get the names, and when describing you get a lot more.  I don't see us distinguishing between listing vs. describing in terms of data -- we are going to send back the same two things (mechanism and iterations) regardless.  So we would probably be talking about creating a `describeUserScramCredentials()` API and a `describeAllUserScramCredentials()` API with the first taking a list and returning a map of futures and the second not taking a list and returning a single future.
   
   But I'm thinking we should just keep it the way it is -- take a possibly-empty list and return a single future regardles of whether the list was empty or not.
   
   Thoughts?




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

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



[GitHub] [kafka] cmccabe commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-685182597


   ok to test


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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469646777



##########
File path: core/src/test/scala/unit/kafka/admin/UserScramCredentialsCommandTest.scala
##########
@@ -0,0 +1,127 @@
+/**
+ * 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 kafka.admin
+
+import java.io.{ByteArrayOutputStream, PrintStream}
+import java.nio.charset.StandardCharsets
+
+import kafka.server.BaseRequestTest
+import kafka.utils.Exit
+import org.junit.Assert._
+import org.junit.Test
+
+class UserScramCredentialsCommandTest extends BaseRequestTest {
+  override def brokerCount = 1
+  var exitStatus: Option[Int] = None
+  var exitMessage: Option[String] = None
+
+  case class ConfigCommandResult(stdout: String, exitStatus: Option[Int] = None)
+
+  private def runConfigCommandViaBroker(args: Array[String]) : ConfigCommandResult = {
+    val byteArrayOutputStream = new ByteArrayOutputStream()
+    val utf8 = StandardCharsets.UTF_8.name
+    val printStream = new PrintStream(byteArrayOutputStream, true, utf8)
+    var exitStatus: Option[Int] = None
+    Exit.setExitProcedure { (status, _) =>

Review comment:
       Hmm, good point, I think there may be a problem here in general because there is only a single exit procedure that can be set globally, and multiple tests that set/reset it in parallel will collide.  There are 16 Scala test classes in `core` out of 260 that do this -- so 6% of test classes.  So I think this will introduce some flakiness to these 16 tests.  Does this sound correct to you, and should we open a separate ticket for this as opposed to trying to fix it here?




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

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



[GitHub] [kafka] cmccabe merged pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe merged pull request #9032:
URL: https://github.com/apache/kafka/pull/9032


   


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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r478656313



##########
File path: tests/kafkatest/version.py
##########
@@ -63,8 +63,13 @@ def reassign_partitions_command_supports_bootstrap_server(self):
         return self >= V_2_5_0
 
     def kafka_configs_command_uses_bootstrap_server(self):
+        # everything except User SCRAM Credentials (KIP-554)
         return self >= V_2_6_0
 
+    def kafka_configs_command_uses_bootstrap_server_scram(self):
+        # User SCRAM Credentials (KIP-554)
+        return self > LATEST_2_6

Review comment:
       I don't think this is quite right.  It should be `self >= V_2_7_0`, right?  And we should add 2.7 to the `version.py` file.  
   
   master branch is already identifying itself as 2.7 anyway.
   
   ```
   [cmccabe@zeratul kafka2]$ git grep 2.7.0 | tail -n 10
   docs/upgrade.html:<h5><a id="upgrade_270_notable" href="#upgrade_270_notable">Notable changes in 2.7.0</a></h5>
   gradle.properties:version=2.7.0-SNAPSHOT
   kafka-merge-pr.py:DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "2.7.0")
   streams/quickstart/java/pom.xml:        <version>2.7.0-SNAPSHOT</version>
   streams/quickstart/java/src/main/resources/archetype-resources/pom.xml:        <kafka.version>2.7.0-SNAPSHOT</kafka.version>
   streams/quickstart/pom.xml:    <version>2.7.0-SNAPSHOT</version>
   streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java:     * @deprecated Since 2.7.0; use {@link #addGlobalStore(StoreBuilder, String, Consumed, ProcessorSupplier)} instead.
   streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala:    "2.7.0"
   tests/kafkatest/__init__.py:__version__ = '2.7.0.dev0'
   tests/kafkatest/version.py:DEV_VERSION = KafkaVersion("2.7.0-SNAPSHOT")
   ```
   
   Similarly `supports_tls_to_zookeeper` should be checking for 2.5 and later, not "after 2.4" (they should be the same, but what if someone messed up incrementing LATEST_2_4, etc....)  It's always better to be explicit about what version something appeared in than to say it happened "after (but not including) some version X"




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r478646271



##########
File path: tests/kafkatest/services/security/security_config.py
##########
@@ -276,18 +287,19 @@ def setup_node(self, node):
         if java_version(node) <= 11 and self.properties.get('tls.version') == 'TLSv1.3':
             self.properties.update({'tls.version': 'TLSv1.2'})
 
-    def setup_credentials(self, node, path, zk_connect, broker):
-        if broker:
-            self.maybe_create_scram_credentials(node, zk_connect, path, self.interbroker_sasl_mechanism,
+    def setup_credentials(self, node, path, connect, creating_broker_user):

Review comment:
       I think we should just have a separate function for this, since there's no code shared between when creating_broker_user=True and when it is False.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472570823



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       > a list RPC to show everything, and a describe RPC to show only some things.
   
   Do you mean a list RPC that takes no arguments and returns every credential defined for every users and a describe RPC that takes 1 or more users and returns every credential defined for those specified users, and they both return the same information for each credential?
   
   Or do you mean a list RPC and a describe RPC that return different sets of information (as is done with list vs. describe topics)?  I think you mean the former (two RPCs, each returning the same thing), but I want to be certain I understand.
   
   > There are a few reasons why. One is that even though we currently only make one RPC, in the future we might make more than one. In that case we would want multiple futures.
   
   I don't understand what this is referring to.  By "we currently only make one RPC" to what are you referring?
   
   > I also feel like in AdminClient, errors should be handled with futures pretty much all the time
   
   Agreed.  Will convert to using futures always, whenever we arrive at the final decision on what the RPCs need to look like.
   
   I'm wondering if we convert to returning futures everywhere, can we stick with the one describe RPC?  For example, could the admin client return a `Future<Map<String, Future<UserScramCredentialDescription>>>`?  Would that work, and if so, would that be a reasonable way to proceed?




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472498809



##########
File path: clients/src/test/java/org/apache/kafka/common/protocol/ProtoUtilsTest.java
##########
@@ -26,11 +26,18 @@
     public void testDelayedAllocationSchemaDetection() throws Exception {
         //verifies that schemas known to retain a reference to the underlying byte buffer are correctly detected.
         for (ApiKeys key : ApiKeys.values()) {
-            if (key == ApiKeys.PRODUCE || key == ApiKeys.JOIN_GROUP || key == ApiKeys.SYNC_GROUP || key == ApiKeys.SASL_AUTHENTICATE
-                || key == ApiKeys.EXPIRE_DELEGATION_TOKEN || key == ApiKeys.RENEW_DELEGATION_TOKEN) {
-                assertTrue(key + " should require delayed allocation", key.requiresDelayedAllocation);
-            } else {
-                assertFalse(key + " should not require delayed allocation", key.requiresDelayedAllocation);
+            switch (key) {
+                case PRODUCE:
+                case JOIN_GROUP:
+                case SYNC_GROUP:
+                case SASL_AUTHENTICATE:
+                case EXPIRE_DELEGATION_TOKEN:
+                case RENEW_DELEGATION_TOKEN:
+                case ALTER_USER_SCRAM_CREDENTIALS:
+                    assertTrue(key + " should require delayed allocation", key.requiresDelayedAllocation);
+                    break;
+                default:
+                    assertFalse(key + " should not require delayed allocation", key.requiresDelayedAllocation);

Review comment:
       I know it's not strictly necessary, but it would be nice to have a "break" after the default clause too




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469650293



##########
File path: core/src/test/scala/unit/kafka/admin/UserScramCredentialsCommandTest.scala
##########
@@ -0,0 +1,127 @@
+/**
+ * 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 kafka.admin
+
+import java.io.{ByteArrayOutputStream, PrintStream}
+import java.nio.charset.StandardCharsets
+
+import kafka.server.BaseRequestTest
+import kafka.utils.Exit
+import org.junit.Assert._
+import org.junit.Test
+
+class UserScramCredentialsCommandTest extends BaseRequestTest {
+  override def brokerCount = 1
+  var exitStatus: Option[Int] = None
+  var exitMessage: Option[String] = None
+
+  case class ConfigCommandResult(stdout: String, exitStatus: Option[Int] = None)
+
+  private def runConfigCommandViaBroker(args: Array[String]) : ConfigCommandResult = {
+    val byteArrayOutputStream = new ByteArrayOutputStream()
+    val utf8 = StandardCharsets.UTF_8.name
+    val printStream = new PrintStream(byteArrayOutputStream, true, utf8)
+    var exitStatus: Option[Int] = None
+    Exit.setExitProcedure { (status, _) =>

Review comment:
       Actually, I think this may not be an issue since parallel tests in Gradle run in separate processes rather than separate threads.  From https://docs.gradle.org/current/dsl/org.gradle.api.tasks.testing.Test.html: `"Test are always run in (one or more) separate JVMs."`




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468873428



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {

Review comment:
       We're planning on getting rid of ControllerNodeProvider as part of KIP-590, so I think these should all just use `leastLoadedNode`.  Right now they set / fetch stuff in ZK directly anyway.
   
   Yes, it will require an email to the mailing list, but I don't think it's a big change




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r474416243



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<UserScramCredentialsDescriptionResult>> future;
+
+    /**
+     *
+     * @param future the required future representing the result of the call
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<UserScramCredentialsDescriptionResult>> future) {
+        this.future = Objects.requireNonNull(future);
+    }
+
+    /**
+     *
+     * @return the future representing the result of the call
+     */
+    public KafkaFuture<List<UserScramCredentialsDescriptionResult>> future() {

Review comment:
       Based on our offline discussion, I thought we were going to have three methods here:
   ```
   KafkaFuture<Map<String, UserScramCredentialsDescription>> all()
   ```
   Returns all the users that were listed, or throws an exception if there was an error describing any of them.
   
   ```
   KafkaFuture<List<String>> users();
   ```
   Returns the names of all the users that were described, whether the describing was successful or not.
   
   ```
   KafkaFuture<UserScramCredentialDescription> description(String userName);
   ```
   Returns a future describing userName, or RESOURCE_NOT_FOUND if the user was not listed or not found.  




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472688926



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       I thought about this more and I think I see a good way out of this difficulty.  We should just have an accessor method like `userInfo(String name)` that returns a `KafkaFuture<user info>`.  We can dynamically create this future if needed.  Then we can have a single RPC and a single API which is just `describe`




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472494891



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,64 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * <p>Describe all SASL/SCRAM credentials.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials() {
+        return describeUserScramCredentials(null, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials for the given users.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null
+     *              or empty.  A user explicitly specified here that does not have a SCRAM credential will not appear
+     *              in the results.

Review comment:
       @rondagostino : It still says "A user explicitly specified here that does not have a SCRAM credential will not appear in the results".  I thought we agreed that such a user would get an error code?




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480329570



##########
File path: core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala
##########
@@ -42,7 +42,18 @@ class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTes
   override def setUp(): Unit = {
     super.setUp()
     // Create client credentials after starting brokers so that dynamic credential creation is also tested
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
+    createScramCredentialWithScramAdminClient(JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
+    createScramCredentialWithScramAdminClient(JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
+  }
+
+  private def createScramCredentialWithScramAdminClient(user: String, password: String) = {

Review comment:
       It was a goal to eliminate all SCRAM credential creation via ZooKeeper where possible.  The only places that do so after this PR are when credentials have to be created before the brokers are started (i.e. when the inter-broker security protocol is SASL/SCRAM).  This code used to create the credential directly via ZooKeeper, but since it occurs after the brokers start it can use the admin client.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r467191680



##########
File path: core/src/main/scala/kafka/admin/ConfigCommand.scala
##########
@@ -508,7 +563,15 @@ object ConfigCommand extends Config {
 
       val entityStr = (entitySubstr(ClientQuotaEntity.USER) ++ entitySubstr(ClientQuotaEntity.CLIENT_ID)).mkString(", ")
       val entriesStr = entries.asScala.map(e => s"${e._1}=${e._2}").mkString(", ")
-      println(s"Configs for ${entityStr} are ${entriesStr}")
+      println(s"Quota configs for ${entityStr} are ${entriesStr}")
+    }
+    // we describe user SCRAM credentials only when we are not describing client information
+    // and we are not given either --entity-default or --user-defaults
+    if (!entityTypes.contains(ConfigType.Client) && !entityNames.contains("")) {
+      getUserScramCredentialConfigs(adminClient, entityNames).foreach { case (user, description) =>

Review comment:
       I'm adding `core/src/test/scala/unit/kafka/admin/UserScramCredentialsCommandTest.scala` -- let me know if this test covers this case.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472501147



##########
File path: core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
##########
@@ -486,7 +486,9 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
   }
 
   @Test
-  def shouldNotAlterNonQuotaClientConfigUsingBootstrapServer(): Unit = {
+  def shouldNotAlterNonQuotaNonScramUserOrClientConfigUsingBootstrapServer(): Unit = {
+    // when using --bootstrap-server, it should be illegal to alter anything that is not a quota and not a SCRAM credential
+    // for both user and client entities

Review comment:
       sorry, this might be a silly question, but how are these constraints different when using --zookeeper?  should we test that as well




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469457068



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new IllegalArgumentException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite aa exception from a previous upsertion, but we don't really care
+                        // since we just needs to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);

Review comment:
       Actually, we also have to deal with too few/too many iterations as well as empty username.  These, along with empty password, are a class unto themselves -- unacceptable credentials.  So I'm abandoning the specific "`UnacceptablePasswordException`/`Errors.UNACCEPTABLE_PASSWORD`" in favor of "`UnacceptableCredentialException`/`Errors.UNACCEPTABLE_CREDENTIAL`"




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r473276446



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));

Review comment:
       ok




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r466681231



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {

Review comment:
       The KIP says `It will be will be sent to the controller, and will return NOT_CONTROLLER if the receiving broker is not the controller.` It says this for both Describe and Alter.  I agree it doesn't seem necessary for Describe.  Would it require an email to the list for notification if we decide to change it?  Or would a KIP update be sufficient?  Do you think we should change it?  @cmccabe any thoughts here as well?




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469476195



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {

Review comment:
       Ok, switching Describe to **not** require that it be done on the controller.
   
   > planning on getting rid of ControllerNodeProvider as part of KIP-590 
   
   Leaving Alter alone for now under the assumption that we will fix it as part of the KIP-590 PR.  Let me know if you wish me to change this now instead.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469637266



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UserScramCredentialUpsertion.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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 java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import java.util.Objects;
+
+/**
+ * A request to update/insert a SASL/SCRAM credential for a user.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class UserScramCredentialUpsertion extends UserScramCredentialAlteration {
+    private final ScramCredentialInfo info;
+    private final byte[] salt;
+    private final byte[] password;
+
+    /**
+     * Constructor that generates a random salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, String password) {
+        this(user, credentialInfo, password.getBytes(StandardCharsets.UTF_8));
+    }
+
+    /**
+     * Constructor that generates a random salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, byte[] password) {
+        this(user, credentialInfo, password, generateRandomSalt());
+    }
+
+    /**
+     * Constructor that accepts an explicit salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     * @param salt the salt to be used
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, byte[] password, byte[] salt) {
+        super(Objects.requireNonNull(user));
+        this.info = Objects.requireNonNull(credentialInfo);
+        this.password = Objects.requireNonNull(password);
+        this.salt = Objects.requireNonNull(salt);
+    }
+
+    /**
+     *
+     * @return the mechanism and iterations
+     */
+    public ScramCredentialInfo credentialInfo() {
+        return info;
+    }
+
+    /**
+     *
+     * @return the salt
+     */
+    public byte[] salt() {
+        return salt;
+    }
+
+    /**
+     *
+     * @return the password
+     */
+    public byte[] password() {
+        return password;
+    }
+
+    private static byte[] generateRandomSalt() {
+        return new BigInteger(130, new SecureRandom()).toString(Character.MAX_RADIX).getBytes(StandardCharsets.UTF_8);

Review comment:
       @cmccabe  I think the approach you suggest leaves out how to identify `length` which itself needs to be randomized.  I got the current implementation from `org.apache.kafka.common.security.scram.internals.ScramFormatter`.  I would have invoked `ScramFormatter.secureRandomBytes()` directly, but it is not `static` and I did not want to either instantiate an instance or change methods to static (though the class is internal, so I could have done that). I instead replicated the logic here.  The array length ends up being random with this approach, as do the bytes in the array.  Let me know what you think.  Currently I've left this as-is.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468885795



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UserScramCredentialUpsertion.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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 java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import java.util.Objects;
+
+/**
+ * A request to update/insert a SASL/SCRAM credential for a user.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class UserScramCredentialUpsertion extends UserScramCredentialAlteration {
+    private final ScramCredentialInfo info;
+    private final byte[] salt;
+    private final byte[] password;
+
+    /**
+     * Constructor that generates a random salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, String password) {
+        this(user, credentialInfo, password.getBytes(StandardCharsets.UTF_8));
+    }
+
+    /**
+     * Constructor that generates a random salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, byte[] password) {
+        this(user, credentialInfo, password, generateRandomSalt());
+    }
+
+    /**
+     * Constructor that accepts an explicit salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     * @param salt the salt to be used
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, byte[] password, byte[] salt) {
+        super(Objects.requireNonNull(user));
+        this.info = Objects.requireNonNull(credentialInfo);
+        this.password = Objects.requireNonNull(password);
+        this.salt = Objects.requireNonNull(salt);
+    }
+
+    /**
+     *
+     * @return the mechanism and iterations
+     */
+    public ScramCredentialInfo credentialInfo() {
+        return info;
+    }
+
+    /**
+     *
+     * @return the salt
+     */
+    public byte[] salt() {
+        return salt;
+    }
+
+    /**
+     *
+     * @return the password
+     */
+    public byte[] password() {
+        return password;
+    }
+
+    private static byte[] generateRandomSalt() {
+        return new BigInteger(130, new SecureRandom()).toString(Character.MAX_RADIX).getBytes(StandardCharsets.UTF_8);

Review comment:
       I think we can avoid a trip through BigInteger with something like this:
   ```
         SecureRandom random = new SecureRandom();
         byte bytes[] = new byte[length];
         random.nextBytes(bytes);
   ```




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468872733



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       If we're going to have a per-user error, then we need `Map<String, KafkaFuture<...>>`.  This will also be useful if we need to add more per-user errors in the future.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r475771230



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<String>> usersFuture;
+    private final Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures;
+
+    /**
+     *
+     * @param usersFuture the future indicating the users described by the call
+     * @param perUserFutures the required map of user names to futures representing the results of describing each
+     *                       user's SCRAM credentials.
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<String>> usersFuture,
+                                              Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures) {
+        this.usersFuture = Objects.requireNonNull(usersFuture);
+        this.perUserFutures = Objects.requireNonNull(perUserFutures);
+    }
+
+    /**
+     *
+     * @return a future for the results of all requested (either explicitly or implicitly via describe-all) users.
+     * The future will complete successfully only if the users future first completes successfully and then all the
+     * futures for the user descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(users());
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            KafkaFuture<Void> succeedsOnlyIfAllDescriptionsSucceed = KafkaFuture.allOf(perUserFutures.values().toArray(
+                    new KafkaFuture[perUserFutures.size()]));
+            KafkaFuture<Map<String, UserScramCredentialsDescription>> mapFuture = succeedsOnlyIfAllDescriptionsSucceed.thenApply(void2 ->
+                perUserFutures.entrySet().stream().collect(Collectors.toMap(
+                    e -> e.getKey(),
+                    e -> valueFromFutureGuaranteedToSucceedAtThisPoint(e.getValue()))));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the map, but we have to return a map at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(mapFuture);
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that were requested (either explicitly or implicitly via
+     * describe-all).  The future will not complete successfully if the user is not authorized to perform the describe
+     * operation; otherwise, it will complete successfully as long as the list of users with credentials can be
+     * successfully determined within some hard-coded timeout period.
+     */
+    public KafkaFuture<List<String>> users() {
+        return usersFuture;
+    }
+
+    /**
+     *
+     * @param userName the name of the user description being requested
+     * @return a future indicating the description results for the given user. The future will complete exceptionally if
+     * the future returned by {@link #users()} completes exceptionally.  If the given user does not exist in the list
+     * of requested users then the future will complete exceptionally with
+     * {@link org.apache.kafka.common.errors.ResourceNotFoundException}.
+     */
+    public KafkaFuture<UserScramCredentialsDescription> description(String userName) {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(usersFuture);
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            // it is possible that there is no future for this user (for example, the original describe request was for
+            // users 1, 2, and 3 but this is looking for user 4), so explicitly take care of that case
+            KafkaFuture<UserScramCredentialsDescription> requestedUserFuture = perUserFutures.get(userName);
+            if (requestedUserFuture == null) {
+                throw new ResourceNotFoundException("No such user: " + userName);
+            }
+            KafkaFuture<Void> succeedsOnlyIfRequestedUserFutureSucceeds = KafkaFuture.allOf(requestedUserFuture);
+            KafkaFuture<UserScramCredentialsDescription> descriptionFuture = succeedsOnlyIfRequestedUserFutureSucceeds.thenApply(void2 ->
+                valueFromFutureGuaranteedToSucceedAtThisPoint(requestedUserFuture));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the description, but we have to return a description at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(descriptionFuture);
+        });
+    }
+
+    private static <T> T valueFromFutureGuaranteedToSucceedAtThisPoint(KafkaFuture<T> future) {
+        try {
+            return future.get();

Review comment:
       > You should not be calling get() here
   
   Looking at the implementation of `KafkaFutureImpl` I agree we should not be calling `get()` because I think it means when the users future completes, the thread that completes it will wait until the results future for that user also completes.  If this is indeed the case then we definitely do not want to be calling `get()` in our code.  One way to make a hard dependency on the users future completing without causing the admin client thread to block as just described is to provide a 2-level results hierarchy as was previously implemented.  This forces the user to complete 2 futures and to do it in their own thread.
   
   I wonder if we have the constructor accept 2 futures rather than 1+N futures we would still create a dependency between the 2 that would block the admin client thread.  Perhaps the second future would have to already have a dependency on the users future built into it.
   
   One thing we have to be careful about is the constraints we put on the arguments to the constructor -- those constraints, whatever they end up being, have to be clearly documented.  If we don't have to impose any -- then that's great.  But if we have to impose/document some, then we should compare that with the 2-level results hierarchy I had provided earlier -- that imposes no constraints whatsoever.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472507947



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       Thinking about this more, I would prefer having both a list RPC to show everything, and a describe RPC to show only some things.  There are a few reasons why.  One is that even though we currently only make one RPC, in the future we might make more than one.  In that case we would want multiple futures.
   
   Another is that the general pattern in Kafka is that list RPCs show everything, but describe RPCs show only some things.  It's true that there are some places where we violate this pattern, but I still think it's worth trying to follow where we can.  Maybe this should be documented better so that when we add new RPCs, people aren't confused about whether to use "list" or "describe."
   
   I also feel like in AdminClient, errors should be handled with futures pretty much all the time, unless there is a really strong reason not to.  This allows people to use an async style of programming.  In contrast, mixing in some errors that aren't futures, but need to be checked explicitly is very likely to confuse people.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r475696554



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<String>> usersFuture;
+    private final Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures;
+
+    /**
+     *
+     * @param usersFuture the future indicating the users described by the call
+     * @param perUserFutures the required map of user names to futures representing the results of describing each
+     *                       user's SCRAM credentials.
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<String>> usersFuture,
+                                              Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures) {
+        this.usersFuture = Objects.requireNonNull(usersFuture);
+        this.perUserFutures = Objects.requireNonNull(perUserFutures);
+    }
+
+    /**
+     *
+     * @return a future for the results of all requested (either explicitly or implicitly via describe-all) users.
+     * The future will complete successfully only if the users future first completes successfully and then all the
+     * futures for the user descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(users());
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            KafkaFuture<Void> succeedsOnlyIfAllDescriptionsSucceed = KafkaFuture.allOf(perUserFutures.values().toArray(
+                    new KafkaFuture[perUserFutures.size()]));
+            KafkaFuture<Map<String, UserScramCredentialsDescription>> mapFuture = succeedsOnlyIfAllDescriptionsSucceed.thenApply(void2 ->
+                perUserFutures.entrySet().stream().collect(Collectors.toMap(
+                    e -> e.getKey(),
+                    e -> valueFromFutureGuaranteedToSucceedAtThisPoint(e.getValue()))));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the map, but we have to return a map at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(mapFuture);
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that were requested (either explicitly or implicitly via
+     * describe-all).  The future will not complete successfully if the user is not authorized to perform the describe
+     * operation; otherwise, it will complete successfully as long as the list of users with credentials can be
+     * successfully determined within some hard-coded timeout period.
+     */
+    public KafkaFuture<List<String>> users() {
+        return usersFuture;
+    }
+
+    /**
+     *
+     * @param userName the name of the user description being requested
+     * @return a future indicating the description results for the given user. The future will complete exceptionally if
+     * the future returned by {@link #users()} completes exceptionally.  If the given user does not exist in the list
+     * of requested users then the future will complete exceptionally with
+     * {@link org.apache.kafka.common.errors.ResourceNotFoundException}.
+     */
+    public KafkaFuture<UserScramCredentialsDescription> description(String userName) {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(usersFuture);
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            // it is possible that there is no future for this user (for example, the original describe request was for
+            // users 1, 2, and 3 but this is looking for user 4), so explicitly take care of that case
+            KafkaFuture<UserScramCredentialsDescription> requestedUserFuture = perUserFutures.get(userName);
+            if (requestedUserFuture == null) {
+                throw new ResourceNotFoundException("No such user: " + userName);
+            }
+            KafkaFuture<Void> succeedsOnlyIfRequestedUserFutureSucceeds = KafkaFuture.allOf(requestedUserFuture);
+            KafkaFuture<UserScramCredentialsDescription> descriptionFuture = succeedsOnlyIfRequestedUserFutureSucceeds.thenApply(void2 ->
+                valueFromFutureGuaranteedToSucceedAtThisPoint(requestedUserFuture));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the description, but we have to return a description at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(descriptionFuture);
+        });
+    }
+
+    private static <T> T valueFromFutureGuaranteedToSucceedAtThisPoint(KafkaFuture<T> future) {
+        try {
+            return future.get();

Review comment:
       You should not be calling `get()` here.
   
   In general it seems like what you really want is a single future behind the scenes that returns everything that you fetched from your single RPC.  Then you just need a few translation functions that pull out the part you need for each future that you're returning to the user.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469410480



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));

Review comment:
       I think `UnsupportedSaslMechanismException` is actually appropriate here -- it already exists and corresponds to `Errors.UNSUPPORTED_SASL_MECHANISM`.
   




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479545936



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<DescribeUserScramCredentialsResponseData> dataFuture;
+
+    /**
+     * Package-private constructor
+     *
+     * @param dataFuture the future indicating response data from the call
+     */
+    DescribeUserScramCredentialsResult(KafkaFuture<DescribeUserScramCredentialsResponseData> dataFuture) {
+        this.dataFuture = Objects.requireNonNull(dataFuture);
+    }
+
+    /**
+     *
+     * @return a future for the results of all described users with map keys (one per user) being consistent with the
+     * contents of the list returned by {@link #users()}. The future will complete successfully only if all such user
+     * descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        return KafkaFuture.allOf(dataFuture).thenApply(v -> {

Review comment:
       There's no reason to call `allOf` on a single future.  After all, `allOf`'s function is convert multiple futures to a single one.  But if you already have a single future, this is not needed.
   
   You could use `dataFuture.thenApply`, but that will not trigger if `dataFuture` is completed exceptionally.
   
   Instead, what you want here is something like the following:
   `dataFuture.whenComplete( (data, exception) -> if (exception != null) { ... error ... }  else { ... handle 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.

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



[GitHub] [kafka] rajinisivaram commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r467052269



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {

Review comment:
       Let's see what @cmccabe thinks. We can do whatever we intend to do for similar describe APIs in the KIP-500 world. If we are changing, then we should update the KIP and send a note to the KIP discussion thread.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468871887



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,64 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * <p>Describe all SASL/SCRAM credentials.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials() {
+        return describeUserScramCredentials(null, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials for the given users.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null
+     *              or empty.  A user explicitly specified here that does not have a SCRAM credential will not appear
+     *              in the results.

Review comment:
       (Another advantage to having a per-user error code is that in the future, we may have more reasons why describing a user might fail, and these might be per-user)




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r470339031



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UserScramCredentialUpsertion.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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 java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import java.util.Objects;
+
+/**
+ * A request to update/insert a SASL/SCRAM credential for a user.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class UserScramCredentialUpsertion extends UserScramCredentialAlteration {
+    private final ScramCredentialInfo info;
+    private final byte[] salt;
+    private final byte[] password;
+
+    /**
+     * Constructor that generates a random salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, String password) {
+        this(user, credentialInfo, password.getBytes(StandardCharsets.UTF_8));
+    }
+
+    /**
+     * Constructor that generates a random salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, byte[] password) {
+        this(user, credentialInfo, password, generateRandomSalt());
+    }
+
+    /**
+     * Constructor that accepts an explicit salt
+     *
+     * @param user the user for which the credential is to be updated/inserted
+     * @param credentialInfo the mechanism and iterations to be used
+     * @param password the password
+     * @param salt the salt to be used
+     */
+    public UserScramCredentialUpsertion(String user, ScramCredentialInfo credentialInfo, byte[] password, byte[] salt) {
+        super(Objects.requireNonNull(user));
+        this.info = Objects.requireNonNull(credentialInfo);
+        this.password = Objects.requireNonNull(password);
+        this.salt = Objects.requireNonNull(salt);
+    }
+
+    /**
+     *
+     * @return the mechanism and iterations
+     */
+    public ScramCredentialInfo credentialInfo() {
+        return info;
+    }
+
+    /**
+     *
+     * @return the salt
+     */
+    public byte[] salt() {
+        return salt;
+    }
+
+    /**
+     *
+     * @return the password
+     */
+    public byte[] password() {
+        return password;
+    }
+
+    private static byte[] generateRandomSalt() {
+        return new BigInteger(130, new SecureRandom()).toString(Character.MAX_RADIX).getBytes(StandardCharsets.UTF_8);

Review comment:
       The latest commit changes the methods on `org.apache.kafka.common.security.scram.internals.ScramFormatter` to be static, and I now reuse that logic.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r474226011



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,64 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * <p>Describe all SASL/SCRAM credentials.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials() {
+        return describeUserScramCredentials(null, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials for the given users.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null
+     *              or empty.  A user explicitly specified here that does not have a SCRAM credential will not appear
+     *              in the results.

Review comment:
       I reworked this Javadoc to list all possible exceptions for both alter and describe.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480324560



##########
File path: core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
##########
@@ -1047,8 +1047,8 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
 
   @Test
   def testAddRemoveSaslListeners(): Unit = {
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword)
+    createScramCredentials(adminClients.head, JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
+    createScramCredentials(adminClients.head, JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword)
     initializeKerberos()

Review comment:
       Good point.  It wasn't waiting before, and it probably didn't/doesn't matter since we were spending time initializing Kerberos, but I added the check anyway just to be safe.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465370586



##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -980,4 +984,137 @@ class AdminManager(val config: KafkaConfig,
       entry.entity -> apiError
     }.toMap
   }
+
+  def describeUserScramCredentials(users: Seq[String]): DescribeUserScramCredentialsResponseData = {
+    val retval = new DescribeUserScramCredentialsResponseData()
+
+    def addToResults(user: String, userConfig: Properties) = {
+      val configKeys = userConfig.stringPropertyNames
+      val hasScramCredential = !ScramMechanism.values().toList.filter(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.toMechanismName)).isEmpty
+      if (hasScramCredential) {
+        val userScramCredentials = new UserScramCredential().setName(user)
+        ScramMechanism.values().foreach(mechanism => if (mechanism != ScramMechanism.UNKNOWN) {
+          val propertyValue = userConfig.getProperty(mechanism.toMechanismName)
+          if (propertyValue != null) {
+            val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
+            userScramCredentials.credentialInfos.add(new CredentialInfo().setMechanism(mechanism.ordinal.toByte).setIterations(iterations))

Review comment:
       I'd really prefer some way to just get the `value` byte out of the ScramMechanism object, rather than relying on the ordering of enum entries.  If we are going to rely on the ordering, that would have to be prominently documented in the class (but let's not....)




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480370463



##########
File path: core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala
##########
@@ -248,4 +250,25 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with
     producerConfig.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
     createProducer()
   }
+
+  private def createScramAdminClient(user: String, password: String): Admin = {

Review comment:
       Ok, I added the following code to SaslSetup, and we implement that first method in the 3 test classes that use this functionality.
   
   ```
     def createPrivilegedAdminClient(): Admin = {
       // create an admin client instance that is authorized to create credentials
       throw new UnsupportedOperationException("Must implement this if a test needs to use it")
     }
   
     def createScramCredentialsViaPrivilegedAdminClient(userName: String, password: String): Unit = {
       val privilegedAdminClient = createPrivilegedAdminClient() // must explicitly implement this method
       try {
         // create the SCRAM credential for the given user
         createScramCredentials(privilegedAdminClient, userName, password)
       } finally {
         privilegedAdminClient.close()
       }
     }
   ```




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480407445



##########
File path: core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
##########
@@ -545,6 +558,16 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
     }
   }
 
+  protected def createScramAdminClient(scramMechanism: String, user: String, password: String): Admin = {

Review comment:
       Ok, this now invokes a new method on SaslSetup().




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465378122



##########
File path: core/src/main/scala/kafka/admin/ConfigCommand.scala
##########
@@ -365,45 +368,90 @@ object ConfigCommand extends Config {
         adminClient.incrementalAlterConfigs(Map(configResource -> alterLogLevelEntries).asJava, alterOptions).all().get(60, TimeUnit.SECONDS)
 
       case ConfigType.User | ConfigType.Client =>
-        val nonQuotaConfigsToAdd = configsToBeAdded.keys.filterNot(QuotaConfigs.isQuotaConfig)
-        if (nonQuotaConfigsToAdd.nonEmpty)
-          throw new IllegalArgumentException(s"Only quota configs can be added for '$entityTypeHead' using --bootstrap-server. Unexpected config names: $nonQuotaConfigsToAdd")
-        val nonQuotaConfigsToDelete = configsToBeDeleted.filterNot(QuotaConfigs.isQuotaConfig)
-        if (nonQuotaConfigsToDelete.nonEmpty)
-          throw new IllegalArgumentException(s"Only quota configs can be deleted for '$entityTypeHead' using --bootstrap-server. Unexpected config names: $nonQuotaConfigsToDelete")
-
-
-        val oldConfig = getClientQuotasConfig(adminClient, entityTypes, entityNames)
-
-        val invalidConfigs = configsToBeDeleted.filterNot(oldConfig.contains)
-        if (invalidConfigs.nonEmpty)
-          throw new InvalidConfigurationException(s"Invalid config(s): ${invalidConfigs.mkString(",")}")
-
-        val alterEntityTypes = entityTypes.map { entType =>
-          entType match {
-            case ConfigType.User => ClientQuotaEntity.USER
-            case ConfigType.Client => ClientQuotaEntity.CLIENT_ID
-            case _ => throw new IllegalArgumentException(s"Unexpected entity type: ${entType}")
+        val hasQuotaConfigsToAdd = configsToBeAdded.keys.exists(QuotaConfigs.isQuotaConfig)
+        val scramConfigsToAddMap = configsToBeAdded.filter(entry => ScramMechanism.isScram(entry._1))
+        val unknownConfigsToAdd = configsToBeAdded.keys.filterNot(key => ScramMechanism.isScram(key) || QuotaConfigs.isQuotaConfig(key))
+        val hasQuotaConfigsToDelete = configsToBeDeleted.exists(QuotaConfigs.isQuotaConfig)
+        val scramConfigsToDelete = configsToBeDeleted.filter(ScramMechanism.isScram)
+        val unknownConfigsToDelete = configsToBeDeleted.filterNot(key => ScramMechanism.isScram(key) || QuotaConfigs.isQuotaConfig(key))
+        if (entityTypeHead == ConfigType.Client || entityTypes.size == 2) { // size==2 for case where users is specified first on the command line, before clients
+          // either just a client or both a user and a client
+          if (unknownConfigsToAdd.nonEmpty || scramConfigsToAddMap.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota configs can be added for '${ConfigType.Client}' using --bootstrap-server. Unexpected config names: ${unknownConfigsToAdd ++ scramConfigsToAddMap.keys}")
+          if (unknownConfigsToDelete.nonEmpty || scramConfigsToDelete.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota configs can be deleted for '${ConfigType.Client}' using --bootstrap-server. Unexpected config names: ${unknownConfigsToDelete ++ scramConfigsToDelete}")
+        } else { // ConfigType.User
+          if (unknownConfigsToAdd.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota and SCRAM credential configs can be added for '${ConfigType.User}' using --bootstrap-server. Unexpected config names: $unknownConfigsToAdd")
+          if (unknownConfigsToDelete.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota and SCRAM credential configs can be deleted for '${ConfigType.User}' using --bootstrap-server. Unexpected config names: $unknownConfigsToDelete")
+          if (scramConfigsToAddMap.nonEmpty || scramConfigsToDelete.nonEmpty) {
+            if (entityNames.exists(_.isEmpty)) // either --entity-type users --entity-default or --user-defaults
+              throw new IllegalArgumentException("The use of --entity-default or --user-defaults is not allowed with User SCRAM Credentials using --bootstrap-server.")
+            if (hasQuotaConfigsToAdd || hasQuotaConfigsToDelete)
+              throw new IllegalArgumentException(s"Cannot alter both quota and SCRAM credential configs simultaneously for '${ConfigType.User}' using --bootstrap-server.")
           }
         }
-        val alterEntityNames = entityNames.map(en => if (en.nonEmpty) en else null)
 
-        // Explicitly populate a HashMap to ensure nulls are recorded properly.
-        val alterEntityMap = new java.util.HashMap[String, String]
-        alterEntityTypes.zip(alterEntityNames).foreach { case (k, v) => alterEntityMap.put(k, v) }
-        val entity = new ClientQuotaEntity(alterEntityMap)
+        if (hasQuotaConfigsToAdd || hasQuotaConfigsToDelete) {
+          // handle altering client/user quota configs
+          val oldConfig = getClientQuotasConfig(adminClient, entityTypes, entityNames)
 
-        val alterOptions = new AlterClientQuotasOptions().validateOnly(false)
-        val alterOps = (configsToBeAddedMap.map { case (key, value) =>
-          val doubleValue = try value.toDouble catch {
-            case _: NumberFormatException =>
-              throw new IllegalArgumentException(s"Cannot parse quota configuration value for ${key}: ${value}")
+          val invalidConfigs = configsToBeDeleted.filterNot(oldConfig.contains)
+          if (invalidConfigs.nonEmpty)
+            throw new InvalidConfigurationException(s"Invalid config(s): ${invalidConfigs.mkString(",")}")
+
+          val alterEntityTypes = entityTypes.map { entType =>
+            entType match {
+              case ConfigType.User => ClientQuotaEntity.USER
+              case ConfigType.Client => ClientQuotaEntity.CLIENT_ID
+              case _ => throw new IllegalArgumentException(s"Unexpected entity type: ${entType}")
+            }
+          }
+          val alterEntityNames = entityNames.map(en => if (en.nonEmpty) en else null)
+
+          // Explicitly populate a HashMap to ensure nulls are recorded properly.
+          val alterEntityMap = new java.util.HashMap[String, String]
+          alterEntityTypes.zip(alterEntityNames).foreach { case (k, v) => alterEntityMap.put(k, v) }
+          val entity = new ClientQuotaEntity(alterEntityMap)
+
+          val alterOptions = new AlterClientQuotasOptions().validateOnly(false)
+          val alterOps = (configsToBeAddedMap.map { case (key, value) =>
+            val doubleValue = try value.toDouble catch {
+              case _: NumberFormatException =>
+                throw new IllegalArgumentException(s"Cannot parse quota configuration value for ${key}: ${value}")
+            }
+            new ClientQuotaAlteration.Op(key, doubleValue)
+          } ++ configsToBeDeleted.map(key => new ClientQuotaAlteration.Op(key, null))).asJavaCollection
+
+          adminClient.alterClientQuotas(Collections.singleton(new ClientQuotaAlteration(entity, alterOps)), alterOptions)
+            .all().get(60, TimeUnit.SECONDS)
+        } else {
+          // handle altering user SCRAM credential configs

Review comment:
       it would be good to split this off into its own function (after verifying that we weren't trying to do both SCRAM + quota) or multiple entity names




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r474416682



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UserScramCredentialsDescription.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.common.errors.ApiException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Representation of all SASL/SCRAM credentials associated with a user that can be retrieved, or an exception indicating
+ * why credentials could not be retrieved.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class UserScramCredentialsDescription {

Review comment:
       As I commented above, I think we should not have the exception here.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472947852



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       Ok, I think I follow now.  You are saying that we could potentially implement describe by invoking 1+N requests to Kafka: one to get the list of credentials (either the list of all of them if we are asking for them all, or the explicitly requested ones we wanted), and then another N requests to get the data for each one.  This on the surface seems like an anti-pattern, but it is not unreasonable for the case where the data is expensive to get in the first place — maybe we are forced to make 1 or more round-trips for each anyway.  So as a general, reusable pattern, yes, I believe it works.
   
   So when we invoke describe, whether it is describe-all or just an explicit few, we return a single future, and that future returns a list of instances (UserName in this case): either 1 instance for each user that has at least 1 credential for the describe-all case, or one instance per distinct user explicitly requested otherwise.  Then each UserName instance has the accessor you mentioned, which in this case returns  Future<UserScramCredentialDescription>.
   
   Do I have that right?




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479552810



##########
File path: core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
##########
@@ -1047,8 +1047,8 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
 
   @Test
   def testAddRemoveSaslListeners(): Unit = {
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
-    createScramCredentials(zkConnect, JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword)
+    createScramCredentials(adminClients.head, JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
+    createScramCredentials(adminClients.head, JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword)
     initializeKerberos()

Review comment:
       Is it necessary to wait for the change to be applied on all brokers after completing the admin client call?




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r470339737



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       I kept the top-level error information but added per-user error information in addition.




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

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



[GitHub] [kafka] rajinisivaram commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r467848781



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/ScramMechanism.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * Representation of a SASL/SCRAM Mechanism.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public enum ScramMechanism {
+    UNKNOWN((byte) 0),
+    SCRAM_SHA_256((byte) 1),
+    SCRAM_SHA_512((byte) 2);
+
+    /**
+     *
+     * @param type the type indicator
+     * @return the instance corresponding to the given type indicator, otherwise {@link #UNKNOWN}
+     */
+    public static ScramMechanism fromType(byte type) {
+        for (ScramMechanism scramMechanism : ScramMechanism.values()) {
+            if (scramMechanism.type == type) {
+                return scramMechanism;
+            }
+        }
+        return UNKNOWN;
+    }
+
+    /**
+     *
+     * @param mechanismName the SASL SCRAM mechanism name
+     * @return the corresponding SASL SCRAM mechanism enum, otherwise {@link #UNKNOWN}
+     * @see <a href="https://tools.ietf.org/html/rfc5802#section-4>
+     *     Salted Challenge Response Authentication Mechanism (SCRAM) SASL and GSS-API Mechanisms, Section 4</a>
+     */
+    public static ScramMechanism fromMechanismName(String mechanismName) {
+        ScramMechanism retvalFoundMechanism = ScramMechanism.valueOf(mechanismName.replace('-', '_'));
+        return retvalFoundMechanism != null ? retvalFoundMechanism : UNKNOWN;
+    }
+
+    /**
+     *
+     * @return the corresponding SASL SCRAM mechanism name
+     * @see <a href="https://tools.ietf.org/html/rfc5802#section-4>
+     *     Salted Challenge Response Authentication Mechanism (SCRAM) SASL and GSS-API Mechanisms, Section 4</a>
+     */
+    public String getMechanismName() {

Review comment:
       We don't use `get` prefix elsewhere, just `mechanismName()`?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,64 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * <p>Describe all SASL/SCRAM credentials.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials() {
+        return describeUserScramCredentials(null, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials for the given users.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null
+     *              or empty.  A user explicitly specified here that does not have a SCRAM credential will not appear
+     *              in the results.

Review comment:
       Should we throw an exception for users which don't exist to be consistent with other APIs?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UserScramCredentialsDescription.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Representation of all SASL/SCRAM credentials associated with a user that can be retrieved.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class UserScramCredentialsDescription {
+    private final String name;
+    private final List<ScramCredentialInfo> infos;
+
+    /**
+     *
+     * @param name the required user name
+     * @param infos the required SASL/SCRAM credential representations for the user
+     */
+    public UserScramCredentialsDescription(String name, List<ScramCredentialInfo> infos) {
+        this.name = Objects.requireNonNull(name);
+        this.infos = Collections.unmodifiableList(new ArrayList<>(Objects.requireNonNull(infos)));
+    }
+
+    /**
+     *
+     * @return the user name
+     */
+    public String getName() {

Review comment:
       Remove `get` prefix

##########
File path: core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala
##########
@@ -0,0 +1,387 @@
+/*
+ * 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 kafka.server
+
+
+import java.nio.charset.StandardCharsets
+import java.util
+import java.util.Properties
+
+import kafka.network.SocketServer
+import kafka.security.authorizer.AclAuthorizer
+import org.apache.kafka.clients.admin.ScramMechanism
+import org.apache.kafka.common.acl.AclOperation
+import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, DescribeUserScramCredentialsRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse, DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
+import org.apache.kafka.common.resource.ResourceType
+import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder}
+import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
+import org.junit.Assert._
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Test AlterUserScramCredentialsRequest/Response API for the cases where either no credentials are altered
+ * or failure is expected due to lack of authorization, sending the request to a non-controller broker, or some other issue.
+ * Also tests the Alter and Describe APIs for the case where credentials are successfully altered/described.
+ */
+class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.ControlledShutdownEnableProp, "false")
+    properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName)
+    properties.put(KafkaConfig.PrincipalBuilderClassProp, classOf[AlterCredentialsTest.TestPrincipalBuilder].getName)
+  }
+
+  @Before
+  override def setUp(): Unit = {
+    AlterCredentialsTest.principal = KafkaPrincipal.ANONYMOUS // default is to be authorized
+    super.setUp()
+  }
+
+  @Test
+  def testAlterNothing(): Unit = {
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterNothingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterSomethingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)))
+        .setUpsertions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_512.getType)))).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(2, results.size)
+    assertTrue("Expected not authorized",
+      results.get(0).errorCode == Errors.CLUSTER_AUTHORIZATION_FAILED.code && results.get(1).errorCode == Errors.CLUSTER_AUTHORIZATION_FAILED.code)
+  }
+
+  @Test
+  def testAlterSameThingTwice(): Unit = {
+    val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+    val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+    val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+      .setIterations(-1).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val upsertion2 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+      .setIterations(-1).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val requests = List (
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion1))
+          .setUpsertions(util.Arrays.asList(upsertion2, upsertion2))).build(),
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion2))
+          .setUpsertions(util.Arrays.asList(upsertion1, upsertion2))).build(),
+    )
+    requests.foreach(request => {
+      val response = sendAlterUserScramCredentialsRequest(request)
+      val results = response.data.results
+      assertEquals(2, results.size)
+      assertTrue("Expected error when altering the same credential twice in a single request",
+        results.get(0).errorCode == Errors.INVALID_REQUEST.code && results.get(1).errorCode == Errors.INVALID_REQUEST.code)
+    })
+  }
+
+  @Test
+  def testAlterEmptyUser(): Unit = {

Review comment:
       Do we test for empty password?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.fromType(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new InvalidRequestException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new InvalidRequestException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite an exception from a previous upsertion, but we don't really care
+                        // since we just need to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);
+                    }
+                });
+        // fail any users immediately that have an illegal alteration as identified above
+        userIllegalAlterationExceptions.entrySet().stream().forEach(entry -> {
+            futures.get(entry.getKey()).completeExceptionally(entry.getValue());
+        });
+
+        // submit alterations for users that do not have an illegal upsertion as identified above
+        Call call = new Call("alterUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public AlterUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new AlterUserScramCredentialsRequest.Builder(
+                        new AlterUserScramCredentialsRequestData().setUpsertions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialUpsertion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(a -> userInsertions.get(a.getUser()).get(((UserScramCredentialUpsertion) a).getInfo().getMechanism()))
+                                .collect(Collectors.toList()))
+                        .setDeletions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialDeletion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(d ->
+                                getScramCredentialDeletion((UserScramCredentialDeletion) d)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                AlterUserScramCredentialsResponse response = (AlterUserScramCredentialsResponse) abstractResponse;
+                // Check for controller change
+                for (Errors error : response.errorCounts().keySet()) {
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                    }
+                }
+                response.data().results().forEach(result -> {
+                    KafkaFutureImpl<Void> future = futures.get(result.user());
+                    if (future == null) {
+                        log.warn("Server response mentioned unknown user {}", result.user());
+                    } else {
+                        Errors error = Errors.forCode(result.errorCode());
+                        if (error != Errors.NONE) {
+                            future.completeExceptionally(error.exception(result.errorMessage()));
+                        } else {
+                            future.complete(null);
+                        }
+                    }
+                });
+                completeUnrealizedFutures(
+                    futures.entrySet().stream(),
+                    user -> "The broker response did not contain a result for user " + user);
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(futures.values(), throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new AlterUserScramCredentialsResult(new HashMap<>(futures));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialUpsertion getScramCredentialUpsertion(UserScramCredentialUpsertion u) throws InvalidKeyException, NoSuchAlgorithmException {
+        AlterUserScramCredentialsRequestData.ScramCredentialUpsertion retval = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion();
+        return retval.setName(u.getUser())
+                .setMechanism(u.getInfo().getMechanism().getType())
+                .setIterations(u.getInfo().getIterations())
+                .setSalt(u.getSalt())
+                .setSaltedPassword(getSaltedPasword(u.getInfo().getMechanism(), u.getPassword(), u.getSalt(), u.getInfo().getIterations()));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialDeletion getScramCredentialDeletion(UserScramCredentialDeletion d) {
+        return new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(d.getUser()).setMechanism(d.getMechanism().getType());
+    }
+
+    private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte[] password, byte[] salt, int iterations) throws NoSuchAlgorithmException, InvalidKeyException {
+        return new ScramFormatter(org.apache.kafka.common.security.scram.internals.ScramMechanism.forMechanismName(publicScramMechanism.getMechanismName()))
+                .hi(password, salt, iterations);

Review comment:
       Iterations can be -1 here? Won't we end up sending a password without applying the salt properly?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/ScramMechanism.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * Representation of a SASL/SCRAM Mechanism.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public enum ScramMechanism {
+    UNKNOWN((byte) 0),
+    SCRAM_SHA_256((byte) 1),
+    SCRAM_SHA_512((byte) 2);
+
+    /**
+     *
+     * @param type the type indicator
+     * @return the instance corresponding to the given type indicator, otherwise {@link #UNKNOWN}
+     */
+    public static ScramMechanism fromType(byte type) {
+        for (ScramMechanism scramMechanism : ScramMechanism.values()) {
+            if (scramMechanism.type == type) {
+                return scramMechanism;
+            }
+        }
+        return UNKNOWN;
+    }
+
+    /**
+     *
+     * @param mechanismName the SASL SCRAM mechanism name
+     * @return the corresponding SASL SCRAM mechanism enum, otherwise {@link #UNKNOWN}
+     * @see <a href="https://tools.ietf.org/html/rfc5802#section-4>
+     *     Salted Challenge Response Authentication Mechanism (SCRAM) SASL and GSS-API Mechanisms, Section 4</a>
+     */
+    public static ScramMechanism fromMechanismName(String mechanismName) {
+        ScramMechanism retvalFoundMechanism = ScramMechanism.valueOf(mechanismName.replace('-', '_'));
+        return retvalFoundMechanism != null ? retvalFoundMechanism : UNKNOWN;
+    }
+
+    /**
+     *
+     * @return the corresponding SASL SCRAM mechanism name
+     * @see <a href="https://tools.ietf.org/html/rfc5802#section-4>
+     *     Salted Challenge Response Authentication Mechanism (SCRAM) SASL and GSS-API Mechanisms, Section 4</a>
+     */
+    public String getMechanismName() {
+        return this.mechanismName;
+    }
+
+    /**
+     *
+     * @return the type indicator for this SASL SCRAM mechanism
+     */
+    public byte getType() {

Review comment:
       As before, `type()`?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/ScramCredentialInfo.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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 java.util.Objects;
+
+/**
+ * Mechanism and iterations for a SASL/SCRAM credential associated with a user.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class ScramCredentialInfo {
+    private final ScramMechanism mechanism;
+    private final int iterations;
+
+    /**
+     *
+     * @param mechanism the required mechanism
+     * @param iterations the number of iterations used when creating the credential
+     */
+    public ScramCredentialInfo(ScramMechanism mechanism, int iterations) {
+        this.mechanism = Objects.requireNonNull(mechanism);
+        this.iterations = iterations;
+    }
+
+    /**
+     *
+     * @return the mechanism
+     */
+    public ScramMechanism getMechanism() {
+        return mechanism;
+    }
+
+    /**
+     *
+     * @return the number of iterations used when creating the credential

Review comment:
       or -1?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UserScramCredentialsDescription.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Representation of all SASL/SCRAM credentials associated with a user that can be retrieved.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class UserScramCredentialsDescription {
+    private final String name;
+    private final List<ScramCredentialInfo> infos;
+
+    /**
+     *
+     * @param name the required user name
+     * @param infos the required SASL/SCRAM credential representations for the user
+     */
+    public UserScramCredentialsDescription(String name, List<ScramCredentialInfo> infos) {
+        this.name = Objects.requireNonNull(name);
+        this.infos = Collections.unmodifiableList(new ArrayList<>(Objects.requireNonNull(infos)));
+    }
+
+    /**
+     *
+     * @return the user name
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     *
+     * @return the unmodifiable list of SASL/SCRAM credential representations for the user
+     */
+    public List<ScramCredentialInfo> getInfos() {

Review comment:
       `credentialInfos()`?

##########
File path: core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala
##########
@@ -0,0 +1,387 @@
+/*
+ * 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 kafka.server
+
+
+import java.nio.charset.StandardCharsets
+import java.util
+import java.util.Properties
+
+import kafka.network.SocketServer
+import kafka.security.authorizer.AclAuthorizer
+import org.apache.kafka.clients.admin.ScramMechanism
+import org.apache.kafka.common.acl.AclOperation
+import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, DescribeUserScramCredentialsRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse, DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
+import org.apache.kafka.common.resource.ResourceType
+import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder}
+import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
+import org.junit.Assert._
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Test AlterUserScramCredentialsRequest/Response API for the cases where either no credentials are altered
+ * or failure is expected due to lack of authorization, sending the request to a non-controller broker, or some other issue.
+ * Also tests the Alter and Describe APIs for the case where credentials are successfully altered/described.
+ */
+class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.ControlledShutdownEnableProp, "false")
+    properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName)
+    properties.put(KafkaConfig.PrincipalBuilderClassProp, classOf[AlterCredentialsTest.TestPrincipalBuilder].getName)
+  }
+
+  @Before
+  override def setUp(): Unit = {
+    AlterCredentialsTest.principal = KafkaPrincipal.ANONYMOUS // default is to be authorized
+    super.setUp()
+  }
+
+  @Test
+  def testAlterNothing(): Unit = {
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterNothingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterSomethingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)))
+        .setUpsertions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_512.getType)))).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(2, results.size)
+    assertTrue("Expected not authorized",
+      results.get(0).errorCode == Errors.CLUSTER_AUTHORIZATION_FAILED.code && results.get(1).errorCode == Errors.CLUSTER_AUTHORIZATION_FAILED.code)
+  }
+
+  @Test
+  def testAlterSameThingTwice(): Unit = {
+    val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+    val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+    val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+      .setIterations(-1).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val upsertion2 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+      .setIterations(-1).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val requests = List (
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion1))
+          .setUpsertions(util.Arrays.asList(upsertion2, upsertion2))).build(),
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion2))
+          .setUpsertions(util.Arrays.asList(upsertion1, upsertion2))).build(),
+    )
+    requests.foreach(request => {
+      val response = sendAlterUserScramCredentialsRequest(request)
+      val results = response.data.results
+      assertEquals(2, results.size)
+      assertTrue("Expected error when altering the same credential twice in a single request",
+        results.get(0).errorCode == Errors.INVALID_REQUEST.code && results.get(1).errorCode == Errors.INVALID_REQUEST.code)

Review comment:
       assertEquals so we know what the error was if test fails? (mutliple places)




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465744271



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));

Review comment:
       How about `InvalidRequestException`?  It's already used in this class, and it might be more appropriate than `InvalidConfigurationException`.




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

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



[GitHub] [kafka] rajinisivaram commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rajinisivaram commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465952166



##########
File path: core/src/main/scala/kafka/admin/ConfigCommand.scala
##########
@@ -508,7 +563,15 @@ object ConfigCommand extends Config {
 
       val entityStr = (entitySubstr(ClientQuotaEntity.USER) ++ entitySubstr(ClientQuotaEntity.CLIENT_ID)).mkString(", ")
       val entriesStr = entries.asScala.map(e => s"${e._1}=${e._2}").mkString(", ")
-      println(s"Configs for ${entityStr} are ${entriesStr}")
+      println(s"Quota configs for ${entityStr} are ${entriesStr}")
+    }
+    // we describe user SCRAM credentials only when we are not describing client information
+    // and we are not given either --entity-default or --user-defaults
+    if (!entityTypes.contains(ConfigType.Client) && !entityNames.contains("")) {
+      getUserScramCredentialConfigs(adminClient, entityNames).foreach { case (user, description) =>

Review comment:
       What do we do if the user is not a SCRAM user? Won't this throw an exception? Can we make sure that user without quota or SCRAM credential doesn't print any errors or exceptions?

##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -980,4 +984,137 @@ class AdminManager(val config: KafkaConfig,
       entry.entity -> apiError
     }.toMap
   }
+
+  def describeUserScramCredentials(users: Option[Seq[String]]): DescribeUserScramCredentialsResponseData = {
+    val retval = new DescribeUserScramCredentialsResponseData()
+
+    def addToResults(user: String, userConfig: Properties) = {
+      val configKeys = userConfig.stringPropertyNames
+      val hasScramCredential = !ScramMechanism.values().toList.filter(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.getMechanismName)).isEmpty
+      if (hasScramCredential) {
+        val userScramCredentials = new UserScramCredential().setName(user)
+        ScramMechanism.values().foreach(mechanism => if (mechanism != ScramMechanism.UNKNOWN) {
+          val propertyValue = userConfig.getProperty(mechanism.getMechanismName)
+          if (propertyValue != null) {
+            val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
+            userScramCredentials.credentialInfos.add(new CredentialInfo().setMechanism(mechanism.getType).setIterations(iterations))
+          }
+        })
+        retval.userScramCredentials.add(userScramCredentials)
+      }
+    }
+
+    if (!users.isDefined || users.get.isEmpty)
+      // describe all users
+      adminZkClient.fetchAllEntityConfigs(ConfigType.User).foreach { case (user, properties) => addToResults(user, properties) }
+    else {
+      // describe specific users
+      // https://stackoverflow.com/questions/24729544/how-to-find-duplicates-in-a-list
+      val duplicatedUsers = users.get.groupBy(identity).collect { case (x, Seq(_, _, _*)) => x }
+      if (duplicatedUsers.nonEmpty) {
+        retval.setError(Errors.INVALID_REQUEST.code())
+        retval.setErrorMessage(s"Cannot describe SCRAM credentials for the same user twice in a single request: ${duplicatedUsers.mkString("[", ", ", "]")}")
+      } else
+        users.get.foreach { user => addToResults(user, adminZkClient.fetchEntityConfig(ConfigType.User, Sanitizer.sanitize(user))) }

Review comment:
       should we catch exception?

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.fromType(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new InvalidRequestException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new InvalidRequestException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite an exception from a previous upsertion, but we don't really care
+                        // since we just need to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);
+                    }
+                });
+        // fail any users immediately that have an illegal alteration as identified above
+        userIllegalAlterationExceptions.entrySet().stream().forEach(entry -> {
+            futures.get(entry.getKey()).completeExceptionally(entry.getValue());
+        });
+
+        // submit alterations for users that do not have an illegal upsertion as identified above
+        Call call = new Call("alterUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public AlterUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new AlterUserScramCredentialsRequest.Builder(
+                        new AlterUserScramCredentialsRequestData().setUpsertions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialUpsertion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(a -> userInsertions.get(a.getUser()).get(((UserScramCredentialUpsertion) a).getInfo().getMechanism()))
+                                .collect(Collectors.toList()))
+                        .setDeletions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialDeletion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(d ->
+                                getScramCredentialDeletion((UserScramCredentialDeletion) d)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                AlterUserScramCredentialsResponse response = (AlterUserScramCredentialsResponse) abstractResponse;
+                // Check for controller change
+                for (Errors error : response.errorCounts().keySet()) {
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                    }
+                }
+                response.data().results().forEach(result -> {
+                    KafkaFutureImpl<Void> future = futures.get(result.user());
+                    if (future == null) {
+                        log.warn("Server response mentioned unknown user {}", result.user());
+                    } else {
+                        Errors error = Errors.forCode(result.errorCode());
+                        if (error != Errors.NONE) {
+                            future.completeExceptionally(error.exception());
+                        } else {
+                            future.complete(null);
+                        }
+                    }
+                });
+                completeUnrealizedFutures(
+                    futures.entrySet().stream(),
+                    user -> "The broker response did not contain a result for user " + user);
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(futures.values(), throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new AlterUserScramCredentialsResult(new HashMap<>(futures));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialUpsertion getScramCredentialUpsertion(UserScramCredentialUpsertion u) throws InvalidKeyException, NoSuchAlgorithmException {
+        AlterUserScramCredentialsRequestData.ScramCredentialUpsertion retval = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion();
+        return retval.setName(u.getUser())
+                .setMechanism(u.getInfo().getMechanism().getType())
+                .setIterations(u.getInfo().getIterations())
+                .setSalt(u.getSalt())
+                .setSaltedPassword(getSaltedPasword(u.getInfo().getMechanism(), u.getPassword(), u.getSalt(), u.getInfo().getIterations()));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialDeletion getScramCredentialDeletion(UserScramCredentialDeletion d) {
+        return new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(d.getUser()).setMechanism(d.getMechanism().getType());
+    }
+
+    private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte[] password, byte[] salt, int interations) throws NoSuchAlgorithmException, InvalidKeyException {

Review comment:
       typo: iterations

##########
File path: core/src/main/scala/kafka/server/KafkaApis.scala
##########
@@ -2973,6 +2975,40 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
   }
 
+  def handleDescribeUserScramCredentialsRequest(request: RequestChannel.Request): Unit = {
+    val describeUserScramCredentialsRequest = request.body[DescribeUserScramCredentialsRequest]
+
+    if (!controller.isActive) {
+      sendResponseMaybeThrottle(request, requestThrottleMs =>
+        describeUserScramCredentialsRequest.getErrorResponse(requestThrottleMs, Errors.NOT_CONTROLLER.exception))
+    } else if (authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME)) {
+      val result = adminManager.describeUserScramCredentials(
+        Option(describeUserScramCredentialsRequest.data.users.asScala.map(_.name).toList))
+      sendResponseMaybeThrottle(request, requestThrottleMs =>
+        new DescribeUserScramCredentialsResponse(result.setThrottleTimeMs(requestThrottleMs)))
+    } else {
+      sendResponseMaybeThrottle(request, requestThrottleMs =>
+        describeUserScramCredentialsRequest.getErrorResponse(requestThrottleMs, Errors.CLUSTER_AUTHORIZATION_FAILED.exception))
+    }
+  }
+
+  def handleAlterUserScramCredentialsRequest(request: RequestChannel.Request): Unit = {
+    val alterUserScramCredentialsRequest = request.body[AlterUserScramCredentialsRequest]
+
+    if (!controller.isActive) {
+      sendResponseMaybeThrottle(request, requestThrottleMs =>
+        alterUserScramCredentialsRequest.getErrorResponse(requestThrottleMs, Errors.NOT_CONTROLLER.exception))
+    } else if (authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) {

Review comment:
       I think we should also not allow users who authenticated using delegation tokens to create or update users. We don't allow these users to create new tokens, it would be odd if they could create a new user or the password of the user of the token.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {

Review comment:
       Does this need to be controller for Describe? It looks similar to DescribeAcls where we use LeastLoadedNode.

##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -980,4 +984,137 @@ class AdminManager(val config: KafkaConfig,
       entry.entity -> apiError
     }.toMap
   }
+
+  def describeUserScramCredentials(users: Option[Seq[String]]): DescribeUserScramCredentialsResponseData = {
+    val retval = new DescribeUserScramCredentialsResponseData()
+
+    def addToResults(user: String, userConfig: Properties) = {
+      val configKeys = userConfig.stringPropertyNames
+      val hasScramCredential = !ScramMechanism.values().toList.filter(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.getMechanismName)).isEmpty
+      if (hasScramCredential) {
+        val userScramCredentials = new UserScramCredential().setName(user)
+        ScramMechanism.values().foreach(mechanism => if (mechanism != ScramMechanism.UNKNOWN) {
+          val propertyValue = userConfig.getProperty(mechanism.getMechanismName)
+          if (propertyValue != null) {
+            val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
+            userScramCredentials.credentialInfos.add(new CredentialInfo().setMechanism(mechanism.getType).setIterations(iterations))
+          }
+        })
+        retval.userScramCredentials.add(userScramCredentials)
+      }
+    }
+
+    if (!users.isDefined || users.get.isEmpty)
+      // describe all users
+      adminZkClient.fetchAllEntityConfigs(ConfigType.User).foreach { case (user, properties) => addToResults(user, properties) }
+    else {
+      // describe specific users
+      // https://stackoverflow.com/questions/24729544/how-to-find-duplicates-in-a-list
+      val duplicatedUsers = users.get.groupBy(identity).collect { case (x, Seq(_, _, _*)) => x }
+      if (duplicatedUsers.nonEmpty) {
+        retval.setError(Errors.INVALID_REQUEST.code())
+        retval.setErrorMessage(s"Cannot describe SCRAM credentials for the same user twice in a single request: ${duplicatedUsers.mkString("[", ", ", "]")}")
+      } else
+        users.get.foreach { user => addToResults(user, adminZkClient.fetchEntityConfig(ConfigType.User, Sanitizer.sanitize(user))) }
+    }
+    retval
+  }
+
+  def alterUserScramCredentials(upsertions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion],
+                                deletions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialDeletion]): AlterUserScramCredentialsResponseData = {
+
+    def scramMechanism(mechanism: Byte): ScramMechanism = {
+      ScramMechanism.fromType(mechanism)
+    }
+
+    def mechanismName(mechanism: Byte): String = {
+      scramMechanism(mechanism).getMechanismName
+    }
+
+    val retval = new AlterUserScramCredentialsResponseData()
+
+    // fail any user that is invalid due to an empty user name, an unknown SCRAM mechanism, or not enough iterations
+    val invalidUsers = (
+      upsertions.filter(upsertion => {
+        if (upsertion.name.isEmpty)
+          true
+        else {
+          val publicScramMechanism = scramMechanism(upsertion.mechanism)
+          publicScramMechanism == ScramMechanism.UNKNOWN ||
+            (upsertion.iterations != -1 &&
+              InternalScramMechanism.forMechanismName(publicScramMechanism.getMechanismName).minIterations > upsertion.iterations)
+        }
+      }).map(_.name) ++ deletions.filter(deletions => deletions.name.isEmpty || scramMechanism(deletions.mechanism) == ScramMechanism.UNKNOWN).map(_.name))
+      .toSet
+    invalidUsers.foreach(user => retval.results.add(new AlterUserScramCredentialsResult().setUser(user)
+      .setErrorCode(Errors.INVALID_REQUEST.code).setErrorMessage("Unknown SCRAM mechanism or too few iterations")))

Review comment:
       Can't we return an exception message that says exactly why, instead of `x or y`

##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -980,4 +984,137 @@ class AdminManager(val config: KafkaConfig,
       entry.entity -> apiError
     }.toMap
   }
+
+  def describeUserScramCredentials(users: Option[Seq[String]]): DescribeUserScramCredentialsResponseData = {
+    val retval = new DescribeUserScramCredentialsResponseData()
+
+    def addToResults(user: String, userConfig: Properties) = {
+      val configKeys = userConfig.stringPropertyNames
+      val hasScramCredential = !ScramMechanism.values().toList.filter(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.getMechanismName)).isEmpty
+      if (hasScramCredential) {
+        val userScramCredentials = new UserScramCredential().setName(user)
+        ScramMechanism.values().foreach(mechanism => if (mechanism != ScramMechanism.UNKNOWN) {
+          val propertyValue = userConfig.getProperty(mechanism.getMechanismName)
+          if (propertyValue != null) {
+            val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
+            userScramCredentials.credentialInfos.add(new CredentialInfo().setMechanism(mechanism.getType).setIterations(iterations))
+          }
+        })
+        retval.userScramCredentials.add(userScramCredentials)
+      }
+    }
+
+    if (!users.isDefined || users.get.isEmpty)
+      // describe all users
+      adminZkClient.fetchAllEntityConfigs(ConfigType.User).foreach { case (user, properties) => addToResults(user, properties) }
+    else {
+      // describe specific users
+      // https://stackoverflow.com/questions/24729544/how-to-find-duplicates-in-a-list
+      val duplicatedUsers = users.get.groupBy(identity).collect { case (x, Seq(_, _, _*)) => x }
+      if (duplicatedUsers.nonEmpty) {
+        retval.setError(Errors.INVALID_REQUEST.code())
+        retval.setErrorMessage(s"Cannot describe SCRAM credentials for the same user twice in a single request: ${duplicatedUsers.mkString("[", ", ", "]")}")
+      } else
+        users.get.foreach { user => addToResults(user, adminZkClient.fetchEntityConfig(ConfigType.User, Sanitizer.sanitize(user))) }
+    }
+    retval
+  }
+
+  def alterUserScramCredentials(upsertions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion],
+                                deletions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialDeletion]): AlterUserScramCredentialsResponseData = {
+
+    def scramMechanism(mechanism: Byte): ScramMechanism = {
+      ScramMechanism.fromType(mechanism)
+    }
+
+    def mechanismName(mechanism: Byte): String = {
+      scramMechanism(mechanism).getMechanismName
+    }
+
+    val retval = new AlterUserScramCredentialsResponseData()
+
+    // fail any user that is invalid due to an empty user name, an unknown SCRAM mechanism, or not enough iterations
+    val invalidUsers = (
+      upsertions.filter(upsertion => {
+        if (upsertion.name.isEmpty)
+          true
+        else {
+          val publicScramMechanism = scramMechanism(upsertion.mechanism)
+          publicScramMechanism == ScramMechanism.UNKNOWN ||
+            (upsertion.iterations != -1 &&
+              InternalScramMechanism.forMechanismName(publicScramMechanism.getMechanismName).minIterations > upsertion.iterations)

Review comment:
       Shouldn't we also limit the max value for iterations?

##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -980,4 +984,137 @@ class AdminManager(val config: KafkaConfig,
       entry.entity -> apiError
     }.toMap
   }
+
+  def describeUserScramCredentials(users: Option[Seq[String]]): DescribeUserScramCredentialsResponseData = {
+    val retval = new DescribeUserScramCredentialsResponseData()
+
+    def addToResults(user: String, userConfig: Properties) = {
+      val configKeys = userConfig.stringPropertyNames
+      val hasScramCredential = !ScramMechanism.values().toList.filter(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.getMechanismName)).isEmpty
+      if (hasScramCredential) {
+        val userScramCredentials = new UserScramCredential().setName(user)
+        ScramMechanism.values().foreach(mechanism => if (mechanism != ScramMechanism.UNKNOWN) {
+          val propertyValue = userConfig.getProperty(mechanism.getMechanismName)
+          if (propertyValue != null) {
+            val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
+            userScramCredentials.credentialInfos.add(new CredentialInfo().setMechanism(mechanism.getType).setIterations(iterations))
+          }
+        })
+        retval.userScramCredentials.add(userScramCredentials)
+      }
+    }
+
+    if (!users.isDefined || users.get.isEmpty)
+      // describe all users
+      adminZkClient.fetchAllEntityConfigs(ConfigType.User).foreach { case (user, properties) => addToResults(user, properties) }
+    else {
+      // describe specific users
+      // https://stackoverflow.com/questions/24729544/how-to-find-duplicates-in-a-list
+      val duplicatedUsers = users.get.groupBy(identity).collect { case (x, Seq(_, _, _*)) => x }
+      if (duplicatedUsers.nonEmpty) {
+        retval.setError(Errors.INVALID_REQUEST.code())
+        retval.setErrorMessage(s"Cannot describe SCRAM credentials for the same user twice in a single request: ${duplicatedUsers.mkString("[", ", ", "]")}")
+      } else
+        users.get.foreach { user => addToResults(user, adminZkClient.fetchEntityConfig(ConfigType.User, Sanitizer.sanitize(user))) }
+    }
+    retval
+  }
+
+  def alterUserScramCredentials(upsertions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion],
+                                deletions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialDeletion]): AlterUserScramCredentialsResponseData = {
+
+    def scramMechanism(mechanism: Byte): ScramMechanism = {
+      ScramMechanism.fromType(mechanism)
+    }
+
+    def mechanismName(mechanism: Byte): String = {
+      scramMechanism(mechanism).getMechanismName
+    }
+
+    val retval = new AlterUserScramCredentialsResponseData()
+
+    // fail any user that is invalid due to an empty user name, an unknown SCRAM mechanism, or not enough iterations
+    val invalidUsers = (
+      upsertions.filter(upsertion => {
+        if (upsertion.name.isEmpty)
+          true
+        else {
+          val publicScramMechanism = scramMechanism(upsertion.mechanism)
+          publicScramMechanism == ScramMechanism.UNKNOWN ||
+            (upsertion.iterations != -1 &&
+              InternalScramMechanism.forMechanismName(publicScramMechanism.getMechanismName).minIterations > upsertion.iterations)
+        }
+      }).map(_.name) ++ deletions.filter(deletions => deletions.name.isEmpty || scramMechanism(deletions.mechanism) == ScramMechanism.UNKNOWN).map(_.name))
+      .toSet
+    invalidUsers.foreach(user => retval.results.add(new AlterUserScramCredentialsResult().setUser(user)
+      .setErrorCode(Errors.INVALID_REQUEST.code).setErrorMessage("Unknown SCRAM mechanism or too few iterations")))
+
+    val initiallyValidUserMechanismPairs = (upsertions.filter(upsertion => !invalidUsers.contains(upsertion.name)).map(upsertion => (upsertion.name, upsertion.mechanism)) ++
+      deletions.filter(deletion => !invalidUsers.contains(deletion.name)).map(deletion => (deletion.name, deletion.mechanism)))
+
+    // https://stackoverflow.com/questions/24729544/how-to-find-duplicates-in-a-list

Review comment:
       Do we include this kind of comment in code?

##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -980,4 +984,137 @@ class AdminManager(val config: KafkaConfig,
       entry.entity -> apiError
     }.toMap
   }
+
+  def describeUserScramCredentials(users: Option[Seq[String]]): DescribeUserScramCredentialsResponseData = {
+    val retval = new DescribeUserScramCredentialsResponseData()
+
+    def addToResults(user: String, userConfig: Properties) = {
+      val configKeys = userConfig.stringPropertyNames
+      val hasScramCredential = !ScramMechanism.values().toList.filter(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.getMechanismName)).isEmpty
+      if (hasScramCredential) {
+        val userScramCredentials = new UserScramCredential().setName(user)
+        ScramMechanism.values().foreach(mechanism => if (mechanism != ScramMechanism.UNKNOWN) {
+          val propertyValue = userConfig.getProperty(mechanism.getMechanismName)
+          if (propertyValue != null) {
+            val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
+            userScramCredentials.credentialInfos.add(new CredentialInfo().setMechanism(mechanism.getType).setIterations(iterations))
+          }
+        })
+        retval.userScramCredentials.add(userScramCredentials)
+      }
+    }
+
+    if (!users.isDefined || users.get.isEmpty)
+      // describe all users
+      adminZkClient.fetchAllEntityConfigs(ConfigType.User).foreach { case (user, properties) => addToResults(user, properties) }
+    else {
+      // describe specific users
+      // https://stackoverflow.com/questions/24729544/how-to-find-duplicates-in-a-list
+      val duplicatedUsers = users.get.groupBy(identity).collect { case (x, Seq(_, _, _*)) => x }
+      if (duplicatedUsers.nonEmpty) {
+        retval.setError(Errors.INVALID_REQUEST.code())
+        retval.setErrorMessage(s"Cannot describe SCRAM credentials for the same user twice in a single request: ${duplicatedUsers.mkString("[", ", ", "]")}")
+      } else
+        users.get.foreach { user => addToResults(user, adminZkClient.fetchEntityConfig(ConfigType.User, Sanitizer.sanitize(user))) }
+    }
+    retval
+  }
+
+  def alterUserScramCredentials(upsertions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion],
+                                deletions: Seq[AlterUserScramCredentialsRequestData.ScramCredentialDeletion]): AlterUserScramCredentialsResponseData = {
+
+    def scramMechanism(mechanism: Byte): ScramMechanism = {
+      ScramMechanism.fromType(mechanism)
+    }
+
+    def mechanismName(mechanism: Byte): String = {
+      scramMechanism(mechanism).getMechanismName
+    }
+
+    val retval = new AlterUserScramCredentialsResponseData()
+
+    // fail any user that is invalid due to an empty user name, an unknown SCRAM mechanism, or not enough iterations
+    val invalidUsers = (
+      upsertions.filter(upsertion => {
+        if (upsertion.name.isEmpty)
+          true
+        else {
+          val publicScramMechanism = scramMechanism(upsertion.mechanism)
+          publicScramMechanism == ScramMechanism.UNKNOWN ||
+            (upsertion.iterations != -1 &&
+              InternalScramMechanism.forMechanismName(publicScramMechanism.getMechanismName).minIterations > upsertion.iterations)

Review comment:
       Shouldn't we also limit the max value for iterations?




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

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



[GitHub] [kafka] rondagostino commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-685977755


   Test failures are flakes/unrelated to this PR, no overlaps across JDKs.


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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479551061



##########
File path: core/src/test/scala/unit/kafka/admin/UserScramCredentialsCommandTest.scala
##########
@@ -0,0 +1,135 @@
+/**
+ * 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 kafka.admin
+
+import java.io.{ByteArrayOutputStream, PrintStream}
+import java.nio.charset.StandardCharsets
+
+import kafka.server.BaseRequestTest
+import kafka.utils.Exit
+import org.junit.Assert._
+import org.junit.Test
+
+class UserScramCredentialsCommandTest extends BaseRequestTest {
+  override def brokerCount = 1
+  var exitStatus: Option[Int] = None
+  var exitMessage: Option[String] = None
+
+  case class ConfigCommandResult(stdout: String, exitStatus: Option[Int] = None)
+
+  private def runConfigCommandViaBroker(args: Array[String]) : ConfigCommandResult = {
+    val byteArrayOutputStream = new ByteArrayOutputStream()
+    val utf8 = StandardCharsets.UTF_8.name
+    val printStream = new PrintStream(byteArrayOutputStream, true, utf8)
+    var exitStatus: Option[Int] = None
+    Exit.setExitProcedure { (status, _) =>
+      exitStatus = Some(status)
+      throw new RuntimeException
+    }
+    try {
+      Console.withOut(printStream) {
+        ConfigCommand.main(Array("--bootstrap-server", brokerList) ++ args)
+      }
+      ConfigCommandResult(byteArrayOutputStream.toString(utf8))
+    } catch {
+      case e: Exception => {

Review comment:
       is it useful to log the exception here at debug level?




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465374713



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new IllegalArgumentException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite aa exception from a previous upsertion, but we don't really care
+                        // since we just needs to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);

Review comment:
       We should have a defined ApiException subclass for this.  NoSuchAlgorithmException could probably be shoehorned into INVALID_CONFIG.  InvalidKeyException probably needs its own defined error code (which we should add to the KIP).
   
   This may seem bureaucratic, but think about it this way: maybe we'll want to do this on the broker side eventually.  Then we need a way to serialize, which plain old java exceptions don't have.




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

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



[GitHub] [kafka] cmccabe commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-679199653


   @rajinisivaram : I'd like to understand your suggestion to forbid authenticating via delegation token here.  It doesn't seem consistent with how we handle delegation tokens in general, so I might be missing something.  It seems like a lot of administrative systems may use delegation tokens and this would make this API not useful for them.


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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472496932



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/UserScramCredentialsDescription.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.common.errors.ApiException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Representation of all SASL/SCRAM credentials associated with a user that can be retrieved, or an exception indicating
+ * why credentials could not be retrieved.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class UserScramCredentialsDescription {
+    private final String name;
+    private final Optional<ApiException> exception;
+    private final List<ScramCredentialInfo> credentialInfos;
+
+    /**
+     * Constructor for when SASL/SCRAM credentials associated with a user could not be retrieved
+     *
+     * @param name the required user name
+     * @param exception the required exception indicating why the credentials for the user could not be retrieved
+     */
+    public UserScramCredentialsDescription(String name, ApiException exception) {
+        this(name, Optional.of(Objects.requireNonNull(exception)), Collections.emptyList());
+    }
+
+    /**
+     * Constructor for when SASL/SCRAM credentials associated with a user are successfully retrieved
+     *
+     * @param name the required user name
+     * @param credentialInfos the required SASL/SCRAM credential representations for the user
+     */
+    public UserScramCredentialsDescription(String name, List<ScramCredentialInfo> credentialInfos) {
+        this(name, Optional.empty(), Objects.requireNonNull(credentialInfos));
+    }
+
+    private UserScramCredentialsDescription(String name, Optional<ApiException> exception, List<ScramCredentialInfo> credentialInfos) {
+        this.name = Objects.requireNonNull(name);
+        this.exception = Objects.requireNonNull(exception);
+        this.credentialInfos = Collections.unmodifiableList(new ArrayList<>(credentialInfos));
+    }
+
+    /**
+     *
+     * @return the user name
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     *
+     * @return the exception, if any, that prevented the user's SASL/SCRAM credentials from being retrieved
+     */
+    public Optional<ApiException> exception() {
+        return exception;
+    }
+
+    /**
+     *
+     * @return the always non-null/unmodifiable list of SASL/SCRAM credential representations for the user
+     * (empty if {@link #exception} defines an exception)
+     */
+    public List<ScramCredentialInfo> credentialInfos() {

Review comment:
       It would be good to throw the exception here if there is one, so that it wasn't possible to ignore the problem




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468882676



##########
File path: core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 kafka.server
+
+
+import java.nio.charset.StandardCharsets
+import java.util
+import java.util.Properties
+
+import kafka.network.SocketServer
+import kafka.security.authorizer.AclAuthorizer
+import org.apache.kafka.clients.admin.ScramMechanism
+import org.apache.kafka.common.acl.AclOperation
+import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, DescribeUserScramCredentialsRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse, DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
+import org.apache.kafka.common.resource.ResourceType
+import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder}
+import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
+import org.junit.Assert._
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Test AlterUserScramCredentialsRequest/Response API for the cases where either no credentials are altered
+ * or failure is expected due to lack of authorization, sending the request to a non-controller broker, or some other issue.
+ * Also tests the Alter and Describe APIs for the case where credentials are successfully altered/described.
+ */
+class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.ControlledShutdownEnableProp, "false")
+    properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName)
+    properties.put(KafkaConfig.PrincipalBuilderClassProp, classOf[AlterCredentialsTest.TestPrincipalBuilder].getName)
+  }
+
+  @Before
+  override def setUp(): Unit = {
+    AlterCredentialsTest.principal = KafkaPrincipal.ANONYMOUS // default is to be authorized
+    super.setUp()
+  }
+
+  @Test
+  def testAlterNothing(): Unit = {
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterNothingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterSomethingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
+        .setUpsertions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)))).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(2, results.size)
+    val msg = "Expected not authorized"
+    assertEquals(msg, Errors.CLUSTER_AUTHORIZATION_FAILED.code, results.get(0).errorCode)
+    assertEquals(msg, Errors.CLUSTER_AUTHORIZATION_FAILED.code, results.get(1).errorCode)
+  }
+
+  @Test
+  def testAlterSameThingTwice(): Unit = {
+    val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+    val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+    val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+      .setIterations(4096).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val upsertion2 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+      .setIterations(4096).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val requests = List (
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion1))
+          .setUpsertions(util.Arrays.asList(upsertion2, upsertion2))).build(),
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion2))
+          .setUpsertions(util.Arrays.asList(upsertion1, upsertion2))).build(),
+    )
+    requests.foreach(request => {
+      val response = sendAlterUserScramCredentialsRequest(request)
+      val results = response.data.results
+      assertEquals(2, results.size)
+      val msg = "Expected error when altering the same credential twice in a single request"
+      assertEquals(msg, Errors.INVALID_REQUEST.code, results.get(0).errorCode)
+      assertEquals(msg, Errors.INVALID_REQUEST.code, results.get(1).errorCode)
+    })
+  }
+
+  @Test
+  def testAlterEmptyUser(): Unit = {
+    val deletionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+    val upsertionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+      .setIterations(4096).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val requests = List (
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletionEmpty))
+          .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build(),
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+          .setUpsertions(util.Arrays.asList(upsertionEmpty))).build(),
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletionEmpty, deletionEmpty))
+          .setUpsertions(util.Arrays.asList(upsertionEmpty))).build(),
+    )
+    requests.foreach(request => {
+      val response = sendAlterUserScramCredentialsRequest(request)
+      val results = response.data.results
+      assertEquals(1, results.size)
+      assertEquals("Expected error when altering an empty user", Errors.INVALID_REQUEST.code, results.get(0).errorCode)
+      assertEquals("\"\" is an illegal user name", results.get(0).errorMessage)
+    })
+  }
+
+  private val user1 = "user1"
+  private val user2 = "user2"
+
+  @Test
+  def testAlterUnknownMechanism(): Unit = {
+    val deletionUnknown1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.UNKNOWN.`type`)
+    val deletionValid1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+    val deletionUnknown2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(10.toByte)
+    val user3 = "user3"
+    val upsertionUnknown3 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user3).setMechanism(ScramMechanism.UNKNOWN.`type`)
+      .setIterations(8192).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val upsertionValid3 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user3).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+      .setIterations(8192).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val user4 = "user4"
+    val upsertionUnknown4 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user4).setMechanism(10.toByte)
+      .setIterations(8192).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val user5 = "user5"
+    val upsertionUnknown5 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user5).setMechanism(ScramMechanism.UNKNOWN.`type`)
+      .setIterations(8192).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val request = new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletionUnknown1, deletionValid1, deletionUnknown2))
+          .setUpsertions(util.Arrays.asList(upsertionUnknown3, upsertionValid3, upsertionUnknown4, upsertionUnknown5))).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+    val results = response.data.results
+    assertEquals(5, results.size)
+    assertEquals("Expected error when altering the credentials with unknown SCRAM mechanisms",
+      0, results.asScala.filterNot(_.errorCode == Errors.INVALID_REQUEST.code).size)
+    results.asScala.foreach(result => assertEquals("Unknown SCRAM mechanism", result.errorMessage))
+  }
+
+  @Test
+  def testAlterTooFewIterations(): Unit = {
+    val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1)
+      .setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(1)
+      .setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Collections.emptyList())
+        .setUpsertions(util.Arrays.asList(upsertionTooFewIterations))).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+    val results = response.data.results
+    assertEquals(1, results.size)
+    assertEquals("Expected error when altering the credentials with too few iterations",
+      0, results.asScala.filterNot(_.errorCode == Errors.INVALID_REQUEST.code).size)
+    assertEquals("Too few iterations", results.get(0).errorMessage)
+  }
+
+  @Test
+  def testAlterTooManyIterations(): Unit = {
+    val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1)
+      .setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(Integer.MAX_VALUE)
+      .setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Collections.emptyList())
+        .setUpsertions(util.Arrays.asList(upsertionTooFewIterations))).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+    val results = response.data.results
+    assertEquals(1, results.size)
+    assertEquals("Expected error when altering the credentials with too many iterations",
+      0, results.asScala.filterNot(_.errorCode == Errors.INVALID_REQUEST.code).size)
+    assertEquals("Too many iterations", results.get(0).errorMessage)
+  }
+
+  @Test
+  def testDeleteSomethingThatDoesNotExist(): Unit = {
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(1, results.size)
+    assertEquals("Expected error when deleting a non-existing credential", Errors.RESOURCE_NOT_FOUND.code, results.get(0).errorCode)
+  }
+
+  @Test
+  def testAlterNotController(): Unit = {
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
+        .setUpsertions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)))).build()
+    val response = sendAlterUserScramCredentialsRequest(request, notControllerSocketServer)
+
+    val results = response.data.results
+    assertEquals(2, results.size)
+    val msg = "Expected controller error when routed incorrectly"
+    assertEquals(msg, Errors.NOT_CONTROLLER.code, results.get(0).errorCode)
+    assertEquals(msg, Errors.NOT_CONTROLLER.code, results.get(1).errorCode)
+  }
+
+  @Test
+  def testAlterAndDescribe(): Unit = {
+    // create a bunch of credentials
+    val request1 = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setUpsertions(util.Arrays.asList(
+          new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion()
+            .setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
+            .setIterations(4096)
+            .setSalt("salt".getBytes(StandardCharsets.UTF_8))
+            .setSaltedPassword("saltedPassword".getBytes(StandardCharsets.UTF_8)),
+          new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion()
+            .setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)
+            .setIterations(8192)
+            .setSalt("salt".getBytes(StandardCharsets.UTF_8))
+            .setSaltedPassword("saltedPassword".getBytes(StandardCharsets.UTF_8)),
+          new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion()
+            .setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)
+            .setIterations(8192)
+            .setSalt("salt".getBytes(StandardCharsets.UTF_8))
+            .setSaltedPassword("saltedPassword".getBytes(StandardCharsets.UTF_8)),
+        ))).build()
+    val response1 = sendAlterUserScramCredentialsRequest(request1)
+    val results1 = response1.data.results
+    assertEquals(2, results1.size)
+    assertEquals("Expected no error when creating the credentials",
+      0, results1.asScala.filterNot(_.errorCode == Errors.NONE.code).size)
+    assertTrue(results1.asScala.exists(_.user == user1))
+    assertTrue(results1.asScala.exists(_.user == user2))
+    // now describe them all
+    val request2 = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response2 = sendDescribeUserScramCredentialsRequest(request2)
+    assertEquals("Expected no error when describing the credentials",
+      Errors.NONE.code, response2.data.error)
+    val results2 = response2.data.userScramCredentials
+    assertEquals(2, results2.size)
+    assertTrue(s"Expected result to contain '$user1' with 2 credentials: $results2",
+      results2.asScala.exists(usc => usc.name == user1 && usc.credentialInfos.size == 2))
+    assertTrue(s"Expected result to contain '$user2' with 1 credential: $results2",
+      results2.asScala.exists(usc => usc.name == user2 && usc.credentialInfos.size == 1))
+    assertTrue(s"Expected result to contain '$user1' with SCRAM_SHA_256/4096 and SCRAM_SHA_512/8192 credentials: $results2",
+      results2.asScala.exists(usc => usc.name == user1 && usc.credentialInfos.asScala.exists(info =>
+        info.mechanism == ScramMechanism.SCRAM_SHA_256.`type` && info.iterations == 4096)
+        && usc.credentialInfos.asScala.exists(info =>
+        info.mechanism == ScramMechanism.SCRAM_SHA_512.`type` && info.iterations == 8192)))
+    assertTrue(s"Expected result to contain '$user2' with SCRAM_SHA_512/8192 credential: $results2",
+      results2.asScala.exists(usc => usc.name == user2 && usc.credentialInfos.asScala.exists(info =>
+        info.mechanism == ScramMechanism.SCRAM_SHA_512.`type` && info.iterations == 8192)))
+    // now describe just one
+    val request3 = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData().setUsers(util.Arrays.asList(
+        new DescribeUserScramCredentialsRequestData.UserName().setName(user1)))).build()
+    val response3 = sendDescribeUserScramCredentialsRequest(request3)
+    assertEquals("Expected no error when describing the credentials",Errors.NONE.code, response3.data.error)
+    val results3 = response3.data.userScramCredentials
+    assertEquals(1, results3.size)
+    assertTrue(s"Expected result to contain '$user1' with 2 credentials: $results3",
+      results3.asScala.exists(usc => usc.name == user1 && usc.credentialInfos.size == 2))
+    assertTrue(s"Expected result to contain '$user1' with SCRAM_SHA_256/4096 and SCRAM_SHA_512/8192 credentials: $results3",
+      results3.asScala.exists(usc => usc.name == user1 && usc.credentialInfos.asScala.exists(info =>
+        info.mechanism == ScramMechanism.SCRAM_SHA_256.`type` && info.iterations == 4096)
+        && usc.credentialInfos.asScala.exists(info =>
+        info.mechanism == ScramMechanism.SCRAM_SHA_512.`type` && info.iterations == 8192)))
+    // now delete a couple of credentials
+    val request4 = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(
+          new AlterUserScramCredentialsRequestData.ScramCredentialDeletion()
+            .setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`),
+          new AlterUserScramCredentialsRequestData.ScramCredentialDeletion()
+            .setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`),
+        ))).build()
+    val response4 = sendAlterUserScramCredentialsRequest(request4)
+    val results4 = response4.data.results
+    assertEquals(2, results4.size)
+    assertEquals("Expected no error when deleting the credentials",
+      0, results4.asScala.filterNot(_.errorCode == Errors.NONE.code).size)
+    assertTrue(s"Expected result to contain '$user1'", results4.asScala.exists(_.user == user1))
+    assertTrue(s"Expected result to contain '$user2'", results4.asScala.exists(_.user == user2))
+    // now describe them all, which should just yield 1 credential
+    val request5 = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response5 = sendDescribeUserScramCredentialsRequest(request5)
+    assertEquals("Expected no error when describing the credentials", Errors.NONE.code, response5.data.error)
+    val results5 = response5.data.userScramCredentials
+    assertEquals(1, results5.size)
+    assertTrue(s"Expected result to contain '$user1' with 1 credential: $results5",
+      results5.asScala.exists(usc => usc.name == user1 && usc.credentialInfos.size == 1))
+    assertTrue(s"Expected result to contain '$user1' with SCRAM_SHA_512/8192 credential: $results5",
+      results5.asScala.exists(usc => usc.name == user1 && usc.credentialInfos.asScala.exists(info =>
+        info.mechanism == ScramMechanism.SCRAM_SHA_512.`type` && info.iterations == 8192)))
+    // now delete the last one
+    val request6 = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(
+          new AlterUserScramCredentialsRequestData.ScramCredentialDeletion()
+            .setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`),
+        ))).build()
+    val response6 = sendAlterUserScramCredentialsRequest(request6)
+    val results6 = response6.data.results
+    assertEquals(1, results6.size)
+    assertEquals("Expected no error when deleting the credentials",
+      0, results4.asScala.filterNot(_.errorCode == Errors.NONE.code).size)
+    assertTrue(s"Expected result to contain '$user1'", results6.asScala.exists(_.user == user1))
+    // now describe them all, which should yield 0 credentials
+    val request7 = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response7 = sendDescribeUserScramCredentialsRequest(request7)
+    assertEquals("Expected no error when describing the credentials",
+      Errors.NONE.code, response7.data.error)
+    val results7 = response7.data.userScramCredentials
+    assertEquals(0, results7.size)
+  }
+
+  private def sendAlterUserScramCredentialsRequest(request: AlterUserScramCredentialsRequest, socketServer: SocketServer = controllerSocketServer): AlterUserScramCredentialsResponse = {
+    connectAndReceive[AlterUserScramCredentialsResponse](request, destination = socketServer)
+  }
+
+  private def sendDescribeUserScramCredentialsRequest(request: DescribeUserScramCredentialsRequest, socketServer: SocketServer = controllerSocketServer): DescribeUserScramCredentialsResponse = {
+    connectAndReceive[DescribeUserScramCredentialsResponse](request, destination = socketServer)
+  }
+}
+
+object AlterCredentialsTest {
+  val UnauthorizedPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "Unauthorized")
+  // Principal used for all client connections. This is modified by tests which
+  // check unauthorized code path
+  var principal = KafkaPrincipal.ANONYMOUS

Review comment:
       Same comment applies here as in the describe test: don't mutate static 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.

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479548121



##########
File path: core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala
##########
@@ -17,6 +17,8 @@
 
 package kafka.server
 
+import java.util

Review comment:
       This should go with the other java imports, which I guess are combined in this file




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r475756645



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<String>> usersFuture;
+    private final Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures;
+
+    /**
+     *
+     * @param usersFuture the future indicating the users described by the call
+     * @param perUserFutures the required map of user names to futures representing the results of describing each
+     *                       user's SCRAM credentials.
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<String>> usersFuture,
+                                              Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures) {
+        this.usersFuture = Objects.requireNonNull(usersFuture);
+        this.perUserFutures = Objects.requireNonNull(perUserFutures);
+    }
+
+    /**
+     *
+     * @return a future for the results of all requested (either explicitly or implicitly via describe-all) users.
+     * The future will complete successfully only if the users future first completes successfully and then all the
+     * futures for the user descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(users());
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            KafkaFuture<Void> succeedsOnlyIfAllDescriptionsSucceed = KafkaFuture.allOf(perUserFutures.values().toArray(
+                    new KafkaFuture[perUserFutures.size()]));
+            KafkaFuture<Map<String, UserScramCredentialsDescription>> mapFuture = succeedsOnlyIfAllDescriptionsSucceed.thenApply(void2 ->
+                perUserFutures.entrySet().stream().collect(Collectors.toMap(
+                    e -> e.getKey(),
+                    e -> valueFromFutureGuaranteedToSucceedAtThisPoint(e.getValue()))));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the map, but we have to return a map at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(mapFuture);
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that were requested (either explicitly or implicitly via

Review comment:
       We've gone back and forth on this.  The KIP does not explicitly state what to do in the case of a describe request for a user that does not have credentials, and we originally coded it to silently drop them, but then we changed it to be consistent with other APIs and raise an error (https://github.com/apache/kafka/pull/9032#discussion_r468871453).  I agree that it isn't totally clear what to do.  Rather than making the change back, I'll leave both this Javadoc and the underlying implementation as-is right now unill we discuss further and decide for sure what we want.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465377717



##########
File path: core/src/main/scala/kafka/admin/ConfigCommand.scala
##########
@@ -365,45 +368,90 @@ object ConfigCommand extends Config {
         adminClient.incrementalAlterConfigs(Map(configResource -> alterLogLevelEntries).asJava, alterOptions).all().get(60, TimeUnit.SECONDS)
 
       case ConfigType.User | ConfigType.Client =>
-        val nonQuotaConfigsToAdd = configsToBeAdded.keys.filterNot(QuotaConfigs.isQuotaConfig)
-        if (nonQuotaConfigsToAdd.nonEmpty)
-          throw new IllegalArgumentException(s"Only quota configs can be added for '$entityTypeHead' using --bootstrap-server. Unexpected config names: $nonQuotaConfigsToAdd")
-        val nonQuotaConfigsToDelete = configsToBeDeleted.filterNot(QuotaConfigs.isQuotaConfig)
-        if (nonQuotaConfigsToDelete.nonEmpty)
-          throw new IllegalArgumentException(s"Only quota configs can be deleted for '$entityTypeHead' using --bootstrap-server. Unexpected config names: $nonQuotaConfigsToDelete")
-
-
-        val oldConfig = getClientQuotasConfig(adminClient, entityTypes, entityNames)
-
-        val invalidConfigs = configsToBeDeleted.filterNot(oldConfig.contains)
-        if (invalidConfigs.nonEmpty)
-          throw new InvalidConfigurationException(s"Invalid config(s): ${invalidConfigs.mkString(",")}")
-
-        val alterEntityTypes = entityTypes.map { entType =>
-          entType match {
-            case ConfigType.User => ClientQuotaEntity.USER
-            case ConfigType.Client => ClientQuotaEntity.CLIENT_ID
-            case _ => throw new IllegalArgumentException(s"Unexpected entity type: ${entType}")
+        val hasQuotaConfigsToAdd = configsToBeAdded.keys.exists(QuotaConfigs.isQuotaConfig)
+        val scramConfigsToAddMap = configsToBeAdded.filter(entry => ScramMechanism.isScram(entry._1))
+        val unknownConfigsToAdd = configsToBeAdded.keys.filterNot(key => ScramMechanism.isScram(key) || QuotaConfigs.isQuotaConfig(key))
+        val hasQuotaConfigsToDelete = configsToBeDeleted.exists(QuotaConfigs.isQuotaConfig)
+        val scramConfigsToDelete = configsToBeDeleted.filter(ScramMechanism.isScram)
+        val unknownConfigsToDelete = configsToBeDeleted.filterNot(key => ScramMechanism.isScram(key) || QuotaConfigs.isQuotaConfig(key))
+        if (entityTypeHead == ConfigType.Client || entityTypes.size == 2) { // size==2 for case where users is specified first on the command line, before clients
+          // either just a client or both a user and a client
+          if (unknownConfigsToAdd.nonEmpty || scramConfigsToAddMap.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota configs can be added for '${ConfigType.Client}' using --bootstrap-server. Unexpected config names: ${unknownConfigsToAdd ++ scramConfigsToAddMap.keys}")
+          if (unknownConfigsToDelete.nonEmpty || scramConfigsToDelete.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota configs can be deleted for '${ConfigType.Client}' using --bootstrap-server. Unexpected config names: ${unknownConfigsToDelete ++ scramConfigsToDelete}")
+        } else { // ConfigType.User
+          if (unknownConfigsToAdd.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota and SCRAM credential configs can be added for '${ConfigType.User}' using --bootstrap-server. Unexpected config names: $unknownConfigsToAdd")
+          if (unknownConfigsToDelete.nonEmpty)
+            throw new IllegalArgumentException(s"Only quota and SCRAM credential configs can be deleted for '${ConfigType.User}' using --bootstrap-server. Unexpected config names: $unknownConfigsToDelete")
+          if (scramConfigsToAddMap.nonEmpty || scramConfigsToDelete.nonEmpty) {
+            if (entityNames.exists(_.isEmpty)) // either --entity-type users --entity-default or --user-defaults
+              throw new IllegalArgumentException("The use of --entity-default or --user-defaults is not allowed with User SCRAM Credentials using --bootstrap-server.")
+            if (hasQuotaConfigsToAdd || hasQuotaConfigsToDelete)
+              throw new IllegalArgumentException(s"Cannot alter both quota and SCRAM credential configs simultaneously for '${ConfigType.User}' using --bootstrap-server.")
           }
         }
-        val alterEntityNames = entityNames.map(en => if (en.nonEmpty) en else null)
 
-        // Explicitly populate a HashMap to ensure nulls are recorded properly.
-        val alterEntityMap = new java.util.HashMap[String, String]
-        alterEntityTypes.zip(alterEntityNames).foreach { case (k, v) => alterEntityMap.put(k, v) }
-        val entity = new ClientQuotaEntity(alterEntityMap)
+        if (hasQuotaConfigsToAdd || hasQuotaConfigsToDelete) {
+          // handle altering client/user quota configs

Review comment:
       it would be good to split this off into its own function (after verifying that we weren't trying to do both SCRAM + quota)




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468149236



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,64 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * <p>Describe all SASL/SCRAM credentials.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials() {
+        return describeUserScramCredentials(null, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials for the given users.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null
+     *              or empty.  A user explicitly specified here that does not have a SCRAM credential will not appear
+     *              in the results.

Review comment:
       Hmm, good question.  The KIP doesn't state what do do here.  @cmccabe thoughts?

##########
File path: core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala
##########
@@ -0,0 +1,387 @@
+/*
+ * 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 kafka.server
+
+
+import java.nio.charset.StandardCharsets
+import java.util
+import java.util.Properties
+
+import kafka.network.SocketServer
+import kafka.security.authorizer.AclAuthorizer
+import org.apache.kafka.clients.admin.ScramMechanism
+import org.apache.kafka.common.acl.AclOperation
+import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, DescribeUserScramCredentialsRequestData}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse, DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
+import org.apache.kafka.common.resource.ResourceType
+import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder}
+import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
+import org.junit.Assert._
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Test AlterUserScramCredentialsRequest/Response API for the cases where either no credentials are altered
+ * or failure is expected due to lack of authorization, sending the request to a non-controller broker, or some other issue.
+ * Also tests the Alter and Describe APIs for the case where credentials are successfully altered/described.
+ */
+class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.ControlledShutdownEnableProp, "false")
+    properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName)
+    properties.put(KafkaConfig.PrincipalBuilderClassProp, classOf[AlterCredentialsTest.TestPrincipalBuilder].getName)
+  }
+
+  @Before
+  override def setUp(): Unit = {
+    AlterCredentialsTest.principal = KafkaPrincipal.ANONYMOUS // default is to be authorized
+    super.setUp()
+  }
+
+  @Test
+  def testAlterNothing(): Unit = {
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterNothingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
+        .setUpsertions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialUpsertion])).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(0, results.size)
+  }
+
+  @Test
+  def testAlterSomethingNotAuthorized(): Unit = {
+    AlterCredentialsTest.principal = AlterCredentialsTest.UnauthorizedPrincipal
+
+    val request = new AlterUserScramCredentialsRequest.Builder(
+      new AlterUserScramCredentialsRequestData()
+        .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)))
+        .setUpsertions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_512.getType)))).build()
+    val response = sendAlterUserScramCredentialsRequest(request)
+
+    val results = response.data.results
+    assertEquals(2, results.size)
+    assertTrue("Expected not authorized",
+      results.get(0).errorCode == Errors.CLUSTER_AUTHORIZATION_FAILED.code && results.get(1).errorCode == Errors.CLUSTER_AUTHORIZATION_FAILED.code)
+  }
+
+  @Test
+  def testAlterSameThingTwice(): Unit = {
+    val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+    val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+    val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name1").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+      .setIterations(-1).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val upsertion2 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("name2").setMechanism(ScramMechanism.SCRAM_SHA_256.getType)
+      .setIterations(-1).setSalt("salt".getBytes).setSaltedPassword("saltedPassword".getBytes)
+    val requests = List (
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion1))
+          .setUpsertions(util.Arrays.asList(upsertion2, upsertion2))).build(),
+      new AlterUserScramCredentialsRequest.Builder(
+        new AlterUserScramCredentialsRequestData()
+          .setDeletions(util.Arrays.asList(deletion1, deletion2))
+          .setUpsertions(util.Arrays.asList(upsertion1, upsertion2))).build(),
+    )
+    requests.foreach(request => {
+      val response = sendAlterUserScramCredentialsRequest(request)
+      val results = response.data.results
+      assertEquals(2, results.size)
+      assertTrue("Expected error when altering the same credential twice in a single request",
+        results.get(0).errorCode == Errors.INVALID_REQUEST.code && results.get(1).errorCode == Errors.INVALID_REQUEST.code)
+    })
+  }
+
+  @Test
+  def testAlterEmptyUser(): Unit = {

Review comment:
       Added a test for it in `UserScramCredentialsCommandTest`.  We can't test for it here because we get a salted password here, and I don't think it is possible for that to be an empty string and allow a successful SASL/SCRAM authentication.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/ScramCredentialInfo.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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 java.util.Objects;
+
+/**
+ * Mechanism and iterations for a SASL/SCRAM credential associated with a user.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public class ScramCredentialInfo {
+    private final ScramMechanism mechanism;
+    private final int iterations;
+
+    /**
+     *
+     * @param mechanism the required mechanism
+     * @param iterations the number of iterations used when creating the credential
+     */
+    public ScramCredentialInfo(ScramMechanism mechanism, int iterations) {
+        this.mechanism = Objects.requireNonNull(mechanism);
+        this.iterations = iterations;
+    }
+
+    /**
+     *
+     * @return the mechanism
+     */
+    public ScramMechanism getMechanism() {
+        return mechanism;
+    }
+
+    /**
+     *
+     * @return the number of iterations used when creating the credential

Review comment:
       TBD.  No change needed if we are getting rid of `-1` as a special value.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.fromType(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new InvalidRequestException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new InvalidRequestException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite an exception from a previous upsertion, but we don't really care
+                        // since we just need to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);
+                    }
+                });
+        // fail any users immediately that have an illegal alteration as identified above
+        userIllegalAlterationExceptions.entrySet().stream().forEach(entry -> {
+            futures.get(entry.getKey()).completeExceptionally(entry.getValue());
+        });
+
+        // submit alterations for users that do not have an illegal upsertion as identified above
+        Call call = new Call("alterUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public AlterUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new AlterUserScramCredentialsRequest.Builder(
+                        new AlterUserScramCredentialsRequestData().setUpsertions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialUpsertion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(a -> userInsertions.get(a.getUser()).get(((UserScramCredentialUpsertion) a).getInfo().getMechanism()))
+                                .collect(Collectors.toList()))
+                        .setDeletions(alterations.stream()
+                                .filter(a -> a instanceof UserScramCredentialDeletion)
+                                .filter(a -> !userIllegalAlterationExceptions.containsKey(a.getUser()))
+                                .map(d ->
+                                getScramCredentialDeletion((UserScramCredentialDeletion) d)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                AlterUserScramCredentialsResponse response = (AlterUserScramCredentialsResponse) abstractResponse;
+                // Check for controller change
+                for (Errors error : response.errorCounts().keySet()) {
+                    if (error == Errors.NOT_CONTROLLER) {
+                        handleNotControllerError(error);
+                    }
+                }
+                response.data().results().forEach(result -> {
+                    KafkaFutureImpl<Void> future = futures.get(result.user());
+                    if (future == null) {
+                        log.warn("Server response mentioned unknown user {}", result.user());
+                    } else {
+                        Errors error = Errors.forCode(result.errorCode());
+                        if (error != Errors.NONE) {
+                            future.completeExceptionally(error.exception(result.errorMessage()));
+                        } else {
+                            future.complete(null);
+                        }
+                    }
+                });
+                completeUnrealizedFutures(
+                    futures.entrySet().stream(),
+                    user -> "The broker response did not contain a result for user " + user);
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                completeAllExceptionally(futures.values(), throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new AlterUserScramCredentialsResult(new HashMap<>(futures));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialUpsertion getScramCredentialUpsertion(UserScramCredentialUpsertion u) throws InvalidKeyException, NoSuchAlgorithmException {
+        AlterUserScramCredentialsRequestData.ScramCredentialUpsertion retval = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion();
+        return retval.setName(u.getUser())
+                .setMechanism(u.getInfo().getMechanism().getType())
+                .setIterations(u.getInfo().getIterations())
+                .setSalt(u.getSalt())
+                .setSaltedPassword(getSaltedPasword(u.getInfo().getMechanism(), u.getPassword(), u.getSalt(), u.getInfo().getIterations()));
+    }
+
+    private static AlterUserScramCredentialsRequestData.ScramCredentialDeletion getScramCredentialDeletion(UserScramCredentialDeletion d) {
+        return new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(d.getUser()).setMechanism(d.getMechanism().getType());
+    }
+
+    private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte[] password, byte[] salt, int iterations) throws NoSuchAlgorithmException, InvalidKeyException {
+        return new ScramFormatter(org.apache.kafka.common.security.scram.internals.ScramMechanism.forMechanismName(publicScramMechanism.getMechanismName()))
+                .hi(password, salt, iterations);

Review comment:
       Yeah, I think there is a fundamental problem with allowing the value `-1` for iterations.  The KIP says "`Note that if the number of iterations is set to -1, the server-side default will be used.`". But we are on the client at this point in the code, and there is no concept for "server-side default" for SCRAM iterations in Kafka.  And unfortunately since we are salting the password here, we need to know the number of iterations.  So I think we need to do either of the following:
   
   1. Add an ability to define server-side default number of iterations per SASL/SCRAM mechanism in Kafka and allow clients to learn them.
   2. Get rid of `-1` as a special value.
   
   It's pretty clear to me that (2) is the way to go, but @rajinisivaram and @cmccabe please chime in.

##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/ScramMechanism.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * Representation of a SASL/SCRAM Mechanism.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public enum ScramMechanism {
+    UNKNOWN((byte) 0),
+    SCRAM_SHA_256((byte) 1),
+    SCRAM_SHA_512((byte) 2);
+
+    /**
+     *
+     * @param type the type indicator
+     * @return the instance corresponding to the given type indicator, otherwise {@link #UNKNOWN}
+     */
+    public static ScramMechanism fromType(byte type) {
+        for (ScramMechanism scramMechanism : ScramMechanism.values()) {
+            if (scramMechanism.type == type) {
+                return scramMechanism;
+            }
+        }
+        return UNKNOWN;
+    }
+
+    /**
+     *
+     * @param mechanismName the SASL SCRAM mechanism name
+     * @return the corresponding SASL SCRAM mechanism enum, otherwise {@link #UNKNOWN}
+     * @see <a href="https://tools.ietf.org/html/rfc5802#section-4>
+     *     Salted Challenge Response Authentication Mechanism (SCRAM) SASL and GSS-API Mechanisms, Section 4</a>
+     */
+    public static ScramMechanism fromMechanismName(String mechanismName) {
+        ScramMechanism retvalFoundMechanism = ScramMechanism.valueOf(mechanismName.replace('-', '_'));
+        return retvalFoundMechanism != null ? retvalFoundMechanism : UNKNOWN;
+    }
+
+    /**
+     *
+     * @return the corresponding SASL SCRAM mechanism name
+     * @see <a href="https://tools.ietf.org/html/rfc5802#section-4>
+     *     Salted Challenge Response Authentication Mechanism (SCRAM) SASL and GSS-API Mechanisms, Section 4</a>
+     */
+    public String getMechanismName() {

Review comment:
       I made the changes here and also removed all `get` prefixes on getters within `ScramCredentialInfo` and `UserScramCredential{Alteration,Deletion,Upsertion,Description}`




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r470101599



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       I don't see the need for multiple futures; everything is returning in a single response from Kafka, and I can't think of why it would be necessary to have some users' credential descriptions available to the API caller but others not.  I'll take a look at keeping a single future for the response but moving the error status from the top level down into each per-user result.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468874664



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new IllegalArgumentException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite aa exception from a previous upsertion, but we don't really care
+                        // since we just needs to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);

Review comment:
       The pattern in Kafka has always been to have very specific error codes and exceptions.  Unfortunately INVALID_REQUEST is documented like this:
   ```
   "This most likely occurs because of a request being malformed by the " +
   "client library or the message was sent to an incompatible broker. See the broker logs " +
   "for more details.",
   ```
   So we probably need a new error code here




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468884545



##########
File path: clients/src/main/java/org/apache/kafka/common/errors/ResourceNotFoundException.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.common.errors;
+
+/**
+ * Exception thrown due to a request for a resource that does not exist.
+ */
+public class ResourceNotFoundException extends ApiException {
+    private final String resource;
+

Review comment:
       We've always set `serialVersionUID` in exception classes, to appease the Java gods.
   ```
       private static final long serialVersionUID = 1L;
   ```
   it's not really necessary since we don't plan on using Java's built-in serialization framework, but I think some tools complain if they don't see it.  And anyway, it's the pattern... for now.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479546505



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<DescribeUserScramCredentialsResponseData> dataFuture;
+
+    /**
+     * Package-private constructor
+     *
+     * @param dataFuture the future indicating response data from the call
+     */
+    DescribeUserScramCredentialsResult(KafkaFuture<DescribeUserScramCredentialsResponseData> dataFuture) {
+        this.dataFuture = Objects.requireNonNull(dataFuture);
+    }
+
+    /**
+     *
+     * @return a future for the results of all described users with map keys (one per user) being consistent with the
+     * contents of the list returned by {@link #users()}. The future will complete successfully only if all such user
+     * descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        return KafkaFuture.allOf(dataFuture).thenApply(v -> {
+            DescribeUserScramCredentialsResponseData data = valueFromFutureGuaranteedToSucceedAtThisPoint(dataFuture);
+            /* Check to make sure every individual described user succeeded.  Note that a successfully described user
+             * is one that appears with *either* a NONE error code or a RESOURCE_NOT_FOUND error code. The
+             * RESOURCE_NOT_FOUND means the client explicitly requested a describe of that particular user but it could
+             * not be described because it does not exist; such a user will not appear as a key in the returned map.
+             */
+            Optional<DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult> optionalFirstFailedDescribe =
+                    data.results().stream().filter(result ->
+                        result.errorCode() != Errors.NONE.code() && result.errorCode() != Errors.RESOURCE_NOT_FOUND.code()).findFirst();
+            if (optionalFirstFailedDescribe.isPresent()) {
+                throw Errors.forCode(optionalFirstFailedDescribe.get().errorCode()).exception(optionalFirstFailedDescribe.get().errorMessage());
+            }
+            Map<String, UserScramCredentialsDescription> retval = new HashMap<>();
+            data.results().stream().forEach(userResult ->
+                    retval.put(userResult.user(), new UserScramCredentialsDescription(userResult.user(),
+                            getScramCredentialInfosFor(userResult))));
+            return retval;
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that meet the request criteria and that have at least one
+     * credential.  The future will not complete successfully if the user is not authorized to perform the describe
+     * operation; otherwise, it will complete successfully as long as the list of users with credentials can be
+     * successfully determined within some hard-coded timeout period. Note that the returned list will not include users
+     * that do not exist/have no credentials: a request to describe an explicit list of users, none of which existed/had
+     * a credential, will result in a future that returns an empty list being returned here. A returned list will
+     * include users that have a credential but that could not be described.
+     */
+    public KafkaFuture<List<String>> users() {
+        return KafkaFuture.allOf(dataFuture).thenApply(v -> {
+            DescribeUserScramCredentialsResponseData data = valueFromFutureGuaranteedToSucceedAtThisPoint(dataFuture);
+            return data.results().stream()
+                    .filter(result -> result.errorCode() != Errors.RESOURCE_NOT_FOUND.code())
+                    .map(result -> result.user()).collect(Collectors.toList());
+        });
+    }
+
+    /**
+     *
+     * @param userName the name of the user description being requested
+     * @return a future indicating the description results for the given user. The future will complete exceptionally if
+     * the future returned by {@link #users()} completes exceptionally.  Note that if the given user does not exist in
+     * the list of described users then the returned future will complete exceptionally with
+     * {@link org.apache.kafka.common.errors.ResourceNotFoundException}.
+     */
+    public KafkaFuture<UserScramCredentialsDescription> description(String userName) {
+        return KafkaFuture.allOf(dataFuture).thenApply(v -> {
+            DescribeUserScramCredentialsResponseData data = valueFromFutureGuaranteedToSucceedAtThisPoint(dataFuture);
+            // it is possible that there is no future for this user (for example, the original describe request was for
+            // users 1, 2, and 3 but this is looking for user 4), so explicitly take care of that case
+            Optional<DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult> optionalUserResult =
+                    data.results().stream().filter(result -> result.user().equals(userName)).findFirst();
+            if (!optionalUserResult.isPresent()) {
+                throw new ResourceNotFoundException("No such user: " + userName);
+            }
+            DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult userResult = optionalUserResult.get();
+            if (userResult.errorCode() != Errors.NONE.code()) {
+                // RESOURCE_NOT_FOUND is included here
+                throw Errors.forCode(userResult.errorCode()).exception(userResult.errorMessage());
+            }
+            return new UserScramCredentialsDescription(userResult.user(), getScramCredentialInfosFor(userResult));
+        });
+    }
+
+    private static List<ScramCredentialInfo> getScramCredentialInfosFor(
+            DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult userResult) {
+        return userResult.credentialInfos().stream().map(c ->
+                new ScramCredentialInfo(ScramMechanism.fromType(c.mechanism()), c.iterations()))
+                .collect(Collectors.toList());
+    }
+
+    private static <T> T valueFromFutureGuaranteedToSucceedAtThisPoint(KafkaFuture<T> future) {
+        try {
+            return future.get();

Review comment:
       The call to `get()` here isn't needed.  See the comment about about using `KafkaFuture#whenComplete`.  Chaining future results is a big part of the power of `CompletableFuture` (which `KafkaFuture` is a clone of... long story...)




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479554911



##########
File path: core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
##########
@@ -545,6 +558,16 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
     }
   }
 
+  protected def createScramAdminClient(scramMechanism: String, user: String, password: String): Admin = {

Review comment:
       can we create a common function in SaslSetup for this?  Seems to be repeated in a lot of tests.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472693268



##########
File path: core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
##########
@@ -486,7 +486,9 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
   }
 
   @Test
-  def shouldNotAlterNonQuotaClientConfigUsingBootstrapServer(): Unit = {
+  def shouldNotAlterNonQuotaNonScramUserOrClientConfigUsingBootstrapServer(): Unit = {
+    // when using --bootstrap-server, it should be illegal to alter anything that is not a quota and not a SCRAM credential
+    // for both user and client entities

Review comment:
       Yeah, I think we should leave it alone for now, for the reasons you state.  But thanks for the context.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472570823



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       > a list RPC to show everything, and a describe RPC to show only some things.
   
   Do you mean a list RPC that takes no arguments and returns every credential defined for every user and a describe RPC that takes 1 or more users and returns every credential defined for those specified users, and they both return the same information for each credential?
   
   Or do you mean a list RPC and a describe RPC that return different sets of information (as is done with list vs. describe topics)?  I think you mean the former (two RPCs, each returning the same thing), but I want to be certain I understand.
   
   > There are a few reasons why. One is that even though we currently only make one RPC, in the future we might make more than one. In that case we would want multiple futures.
   
   I don't understand what this is referring to.  By "we currently only make one RPC" to what are you referring?
   
   > I also feel like in AdminClient, errors should be handled with futures pretty much all the time
   
   Agreed.  Will convert to using futures always, whenever we arrive at the final decision on what the RPCs need to look like.
   
   I'm wondering if we convert to returning futures everywhere, can we stick with the one describe RPC?  For example, could the admin client return a `Future<Map<String, Future<UserScramCredentialDescription>>>`?  Would that work, and if so, would that be a reasonable way to proceed?




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

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



[GitHub] [kafka] rondagostino commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-673778747


   @cmccabe Thanks again for the reviews.  I think all comments are addressed.  Here's where we stand.  I will make the indicated KIP changes and announce them in the DISCUSS email thread once we have the full set of KIP changes agreed to/reviewed in this PR.
   
   1. I removed -1 as a valid iterations value in the code. 
   2. Users who authenticate using delegation tokens are not allowed to alter user SRAM credentials.
   3. It is no longer necessary to send Describe requests to the controller
   4. I added per-user error information in the Describe response, and kept the top-level error information.  The top-level error is non-zero if there is an authorization issue or there is some kind of infrastructure issue.  Otherwise, if some users can't de described but others can, the top-level error indicates success and the per-user errors indicate success or failure accordingly.  Asking to describe a user that doesn't exist or asking to describe the same user twice results in the corresponding user-level error being set.
   5. We've added 3 new errors/exception classes: `RESOURCE_NOT_FOUND/ResourceNotFoundException`, `DUPLICATE_RESOURCE/DuplicateResourceException`, and `UNACCEPTABLE_CREDENTIAL/UnacceptableCredentialException` and we leverage those in the expected places.
   6. We leverage the existing `UNSUPPORTED_SASL_MECHANISM/UnsupportedSaslMechanismException` where it would be expected.
   


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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469646777



##########
File path: core/src/test/scala/unit/kafka/admin/UserScramCredentialsCommandTest.scala
##########
@@ -0,0 +1,127 @@
+/**
+ * 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 kafka.admin
+
+import java.io.{ByteArrayOutputStream, PrintStream}
+import java.nio.charset.StandardCharsets
+
+import kafka.server.BaseRequestTest
+import kafka.utils.Exit
+import org.junit.Assert._
+import org.junit.Test
+
+class UserScramCredentialsCommandTest extends BaseRequestTest {
+  override def brokerCount = 1
+  var exitStatus: Option[Int] = None
+  var exitMessage: Option[String] = None
+
+  case class ConfigCommandResult(stdout: String, exitStatus: Option[Int] = None)
+
+  private def runConfigCommandViaBroker(args: Array[String]) : ConfigCommandResult = {
+    val byteArrayOutputStream = new ByteArrayOutputStream()
+    val utf8 = StandardCharsets.UTF_8.name
+    val printStream = new PrintStream(byteArrayOutputStream, true, utf8)
+    var exitStatus: Option[Int] = None
+    Exit.setExitProcedure { (status, _) =>

Review comment:
       Hmm, good point, I think there may be a problem here in general because there is only a single exit procedure that can be set globally, and multiple tests that set/reset it in parallel will collide.  There are 16 Scala test classes in `core` out of 260 that do this -- so 4% of test classes.  So I think this will introduce some flakiness to these 16 tests.  Does this sound correct to you, and should we open a separate ticket for this as opposed to trying to fix it here?




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468876395



##########
File path: core/src/test/scala/unit/kafka/admin/UserScramCredentialsCommandTest.scala
##########
@@ -0,0 +1,127 @@
+/**
+ * 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 kafka.admin
+
+import java.io.{ByteArrayOutputStream, PrintStream}
+import java.nio.charset.StandardCharsets
+
+import kafka.server.BaseRequestTest
+import kafka.utils.Exit
+import org.junit.Assert._
+import org.junit.Test
+
+class UserScramCredentialsCommandTest extends BaseRequestTest {
+  override def brokerCount = 1
+  var exitStatus: Option[Int] = None
+  var exitMessage: Option[String] = None
+
+  case class ConfigCommandResult(stdout: String, exitStatus: Option[Int] = None)
+
+  private def runConfigCommandViaBroker(args: Array[String]) : ConfigCommandResult = {
+    val byteArrayOutputStream = new ByteArrayOutputStream()
+    val utf8 = StandardCharsets.UTF_8.name
+    val printStream = new PrintStream(byteArrayOutputStream, true, utf8)
+    var exitStatus: Option[Int] = None
+    Exit.setExitProcedure { (status, _) =>

Review comment:
       How does this interact with JUnit running tests concurrently?  cc @ijuma @hachikuji 




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r475698813



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<String>> usersFuture;
+    private final Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures;
+
+    /**
+     *
+     * @param usersFuture the future indicating the users described by the call
+     * @param perUserFutures the required map of user names to futures representing the results of describing each
+     *                       user's SCRAM credentials.
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<String>> usersFuture,
+                                              Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures) {
+        this.usersFuture = Objects.requireNonNull(usersFuture);
+        this.perUserFutures = Objects.requireNonNull(perUserFutures);
+    }
+
+    /**
+     *
+     * @return a future for the results of all requested (either explicitly or implicitly via describe-all) users.
+     * The future will complete successfully only if the users future first completes successfully and then all the
+     * futures for the user descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(users());
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            KafkaFuture<Void> succeedsOnlyIfAllDescriptionsSucceed = KafkaFuture.allOf(perUserFutures.values().toArray(
+                    new KafkaFuture[perUserFutures.size()]));
+            KafkaFuture<Map<String, UserScramCredentialsDescription>> mapFuture = succeedsOnlyIfAllDescriptionsSucceed.thenApply(void2 ->
+                perUserFutures.entrySet().stream().collect(Collectors.toMap(
+                    e -> e.getKey(),
+                    e -> valueFromFutureGuaranteedToSucceedAtThisPoint(e.getValue()))));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the map, but we have to return a map at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(mapFuture);
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that were requested (either explicitly or implicitly via

Review comment:
       I think it would be better to make this "a future indicating the users that were listed" (rather than "requested").  It's maybe a bit of a subtle distinction but think about things like requesting the null user, or the empty string user.  It's awkward to put that here.  I think if we explicitly request a user but it doesn't exist, it should be omitted from here as well.  That gives us more flexibility in the future with the API.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r475750058



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<String>> usersFuture;
+    private final Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures;
+
+    /**
+     *
+     * @param usersFuture the future indicating the users described by the call
+     * @param perUserFutures the required map of user names to futures representing the results of describing each
+     *                       user's SCRAM credentials.
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<String>> usersFuture,
+                                              Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures) {
+        this.usersFuture = Objects.requireNonNull(usersFuture);
+        this.perUserFutures = Objects.requireNonNull(perUserFutures);
+    }
+
+    /**
+     *
+     * @return a future for the results of all requested (either explicitly or implicitly via describe-all) users.
+     * The future will complete successfully only if the users future first completes successfully and then all the
+     * futures for the user descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(users());
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            KafkaFuture<Void> succeedsOnlyIfAllDescriptionsSucceed = KafkaFuture.allOf(perUserFutures.values().toArray(
+                    new KafkaFuture[perUserFutures.size()]));
+            KafkaFuture<Map<String, UserScramCredentialsDescription>> mapFuture = succeedsOnlyIfAllDescriptionsSucceed.thenApply(void2 ->
+                perUserFutures.entrySet().stream().collect(Collectors.toMap(
+                    e -> e.getKey(),
+                    e -> valueFromFutureGuaranteedToSucceedAtThisPoint(e.getValue()))));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the map, but we have to return a map at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(mapFuture);
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that were requested (either explicitly or implicitly via
+     * describe-all).  The future will not complete successfully if the user is not authorized to perform the describe
+     * operation; otherwise, it will complete successfully as long as the list of users with credentials can be
+     * successfully determined within some hard-coded timeout period.
+     */
+    public KafkaFuture<List<String>> users() {
+        return usersFuture;
+    }
+
+    /**
+     *
+     * @param userName the name of the user description being requested
+     * @return a future indicating the description results for the given user. The future will complete exceptionally if
+     * the future returned by {@link #users()} completes exceptionally.  If the given user does not exist in the list
+     * of requested users then the future will complete exceptionally with
+     * {@link org.apache.kafka.common.errors.ResourceNotFoundException}.
+     */
+    public KafkaFuture<UserScramCredentialsDescription> description(String userName) {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(usersFuture);
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            // it is possible that there is no future for this user (for example, the original describe request was for
+            // users 1, 2, and 3 but this is looking for user 4), so explicitly take care of that case
+            KafkaFuture<UserScramCredentialsDescription> requestedUserFuture = perUserFutures.get(userName);
+            if (requestedUserFuture == null) {
+                throw new ResourceNotFoundException("No such user: " + userName);
+            }
+            KafkaFuture<Void> succeedsOnlyIfRequestedUserFutureSucceeds = KafkaFuture.allOf(requestedUserFuture);
+            KafkaFuture<UserScramCredentialsDescription> descriptionFuture = succeedsOnlyIfRequestedUserFutureSucceeds.thenApply(void2 ->
+                valueFromFutureGuaranteedToSucceedAtThisPoint(requestedUserFuture));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the description, but we have to return a description at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(descriptionFuture);
+        });
+    }
+
+    private static <T> T valueFromFutureGuaranteedToSucceedAtThisPoint(KafkaFuture<T> future) {
+        try {
+            return future.get();

Review comment:
       >  a single future behind the scenes that returns everything that you fetched from your single RPC 
   
   That would introduce an assumption that none of the users' results are available until all of the users' results are available.  For example, currently this class accepts two arguments in its constructor: a future to a list of user names, and a map of usernames-to-futures for the results of each user.  If instead for the second argument we accept a single future to a map of usernames-to-results, then that future must succeed all the time (because we need to distinguish between successes and failures for different users).  Each user's result would have to contain the error code, and no user's results will be available to the client until all of the users' results (the underlying map) is available.  It is true that this is how it is implemented today -- no user's results will be available until all users' results are available, and each of the individual results has an error code -- but Is it acceptable to impose such an all-or-nothing result availability constraint?  The idea of fl
 exibility in the API going forward is an ambiguous one, so we should explicitly decide here whether to keep it the way it is now (allowing different users to complete at different times even though it isn't possible right now) vs. changing the constructor and constraining the results for any particular user to be available only after the results for all users are available.  What do you think?




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r478657275



##########
File path: core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala
##########
@@ -169,6 +169,18 @@ object JaasTestUtils {
     jaasFile
   }
 
+  // Returns a SASL/SCRAM configuration using credentials for the given user and password
+  def scramClientLoginModule(mechanism: String, scramUser: String, scramPassword: String): String = {
+    mechanism match {
+      case "SCRAM-SHA-256" | "SCRAM-SHA-512" =>

Review comment:
       Can we not have these be naked constants?  Maybe we could iterate over an enum or something?  If we add another mechanism, it would be nice not to have to manually edit this.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472570823



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       > a list RPC to show everything, and a describe RPC to show only some things.
   
   Do you mean a list RPC that takes no arguments and returns every credential defined for every users and a describe RPC that takes 1 or more users and returns every credential defined for those specified users, and they both return the same information for each credential?
   
   Or do you mean a list RPC and a describe RPC that return different sets of information (as is done with list vs. describe topics)?  I think you mean the former (two RPCs, each returning the same thing), but I want to be certain I understand.
   
   > There are a few reasons why. One is that even though we currently only make one RPC, in the future we might make more than one. In that case we would want multiple futures.
   
   I don't understand what this is referring to.  By "we currently only make one RPC" to what are you referring?
   
   > I also feel like in AdminClient, errors should be handled with futures pretty much all the time
   
   Agreed.  Will convert to using futures always, whenever we arrive at the final decision on what the RPCs need to look like.
   
   I'm wondering if we convert to returning futures everywhere, can we stick with the one describe RPC?  For example, could the admin client return a `Future<Map<String, Future<Void>>>`?  Would that work, and if so, would that be a reasonable way to proceed?




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472530253



##########
File path: core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
##########
@@ -486,7 +486,9 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
   }
 
   @Test
-  def shouldNotAlterNonQuotaClientConfigUsingBootstrapServer(): Unit = {
+  def shouldNotAlterNonQuotaNonScramUserOrClientConfigUsingBootstrapServer(): Unit = {
+    // when using --bootstrap-server, it should be illegal to alter anything that is not a quota and not a SCRAM credential
+    // for both user and client entities

Review comment:
       @cmccabe Good question, actually.  There is already a check to make sure a non-existent config cannot be **deleted** via `--zookeeper`: `shouldNotUpdateConfigIfNonExistingConfigIsDeletedUsingZookeper()`.  This test passes, of course.
   
   However, there is no check to make sure an unrecognized config can be **added**, and in fact if I add that test it fails; the code will gladly go ahead and add anything we wish (and it will gladly go ahead and delete it if we wish as well -- the above test is only checking that something that doesn't exist can't be deleted).
   
   The next question, of course, is whether we should "fix" this or not.  What do you think?  To fix it we would need the full set of allowed configs at the User, Client, Topic, and Broker levels and then insert code to check accordingly.  Since the ZooKeeper update path is going away due to KIP-500, I'm wondering if we can just leave it alone?




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480370634



##########
File path: core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala
##########
@@ -545,6 +558,16 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
     }
   }
 
+  protected def createScramAdminClient(scramMechanism: String, user: String, password: String): Admin = {

Review comment:
       See above




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465369550



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/ScramMechanism.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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;
+
+/**
+ * Representation of a SASL/SCRAM Mechanism.
+ *
+ * @see <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API">KIP-554: Add Broker-side SCRAM Config API</a>
+ */
+public enum ScramMechanism {
+    UNKNOWN((byte) 0),
+    SCRAM_SHA_256((byte) 1),
+    SCRAM_SHA_512((byte) 2);
+
+    /**
+     *
+     * @param b the byte representation
+     * @return the instance corresponding to the given byte value, otherwise {@link #UNKNOWN}
+     */
+    public static ScramMechanism from(byte b) {
+        for (ScramMechanism scramMechanism : ScramMechanism.values()) {
+            if (scramMechanism.type == b) {
+                return scramMechanism;
+            }
+        }
+        return UNKNOWN;
+    }
+
+    /**
+     *
+     * @param mechanismName the SASL SCRAM mechanism name
+     * @return the corresponding SASL SCRAM mechanism enum
+     * @see <a href="https://tools.ietf.org/html/rfc5802#section-4>
+     *     Salted Challenge Response Authentication Mechanism (SCRAM) SASL and GSS-API Mechanisms, Section 4</a>
+     */
+    public static ScramMechanism fromMechanismName(String mechanismName) {

Review comment:
       Hmm... for consistency with `from(byte)`, shouldn't we return UNKNOWN if we don't know the name?




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r475771230



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<String>> usersFuture;
+    private final Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures;
+
+    /**
+     *
+     * @param usersFuture the future indicating the users described by the call
+     * @param perUserFutures the required map of user names to futures representing the results of describing each
+     *                       user's SCRAM credentials.
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<String>> usersFuture,
+                                              Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures) {
+        this.usersFuture = Objects.requireNonNull(usersFuture);
+        this.perUserFutures = Objects.requireNonNull(perUserFutures);
+    }
+
+    /**
+     *
+     * @return a future for the results of all requested (either explicitly or implicitly via describe-all) users.
+     * The future will complete successfully only if the users future first completes successfully and then all the
+     * futures for the user descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(users());
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            KafkaFuture<Void> succeedsOnlyIfAllDescriptionsSucceed = KafkaFuture.allOf(perUserFutures.values().toArray(
+                    new KafkaFuture[perUserFutures.size()]));
+            KafkaFuture<Map<String, UserScramCredentialsDescription>> mapFuture = succeedsOnlyIfAllDescriptionsSucceed.thenApply(void2 ->
+                perUserFutures.entrySet().stream().collect(Collectors.toMap(
+                    e -> e.getKey(),
+                    e -> valueFromFutureGuaranteedToSucceedAtThisPoint(e.getValue()))));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the map, but we have to return a map at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(mapFuture);
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that were requested (either explicitly or implicitly via
+     * describe-all).  The future will not complete successfully if the user is not authorized to perform the describe
+     * operation; otherwise, it will complete successfully as long as the list of users with credentials can be
+     * successfully determined within some hard-coded timeout period.
+     */
+    public KafkaFuture<List<String>> users() {
+        return usersFuture;
+    }
+
+    /**
+     *
+     * @param userName the name of the user description being requested
+     * @return a future indicating the description results for the given user. The future will complete exceptionally if
+     * the future returned by {@link #users()} completes exceptionally.  If the given user does not exist in the list
+     * of requested users then the future will complete exceptionally with
+     * {@link org.apache.kafka.common.errors.ResourceNotFoundException}.
+     */
+    public KafkaFuture<UserScramCredentialsDescription> description(String userName) {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(usersFuture);
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            // it is possible that there is no future for this user (for example, the original describe request was for
+            // users 1, 2, and 3 but this is looking for user 4), so explicitly take care of that case
+            KafkaFuture<UserScramCredentialsDescription> requestedUserFuture = perUserFutures.get(userName);
+            if (requestedUserFuture == null) {
+                throw new ResourceNotFoundException("No such user: " + userName);
+            }
+            KafkaFuture<Void> succeedsOnlyIfRequestedUserFutureSucceeds = KafkaFuture.allOf(requestedUserFuture);
+            KafkaFuture<UserScramCredentialsDescription> descriptionFuture = succeedsOnlyIfRequestedUserFutureSucceeds.thenApply(void2 ->
+                valueFromFutureGuaranteedToSucceedAtThisPoint(requestedUserFuture));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the description, but we have to return a description at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(descriptionFuture);
+        });
+    }
+
+    private static <T> T valueFromFutureGuaranteedToSucceedAtThisPoint(KafkaFuture<T> future) {
+        try {
+            return future.get();

Review comment:
       > You should not be calling get() here
   
   Looking at the implementation of `KafkaFutureImpl` I agree we should not be calling `get()` because I think it means when the users future completes the thread that completes it will wait until the results future for that user also completes.  If this is indeed the case then we definitely do not want to be calling `get()` in our code.  One way to make a hard dependency on the users future completing without causing the admin client thread to block as just described is to provide a 2-level results hierarchy as was previously implemented.  This forces the user to complete 2 futures and to do it in their own thread.
   
   I wonder if we have the constructor accept 2 futures rather than 1+N futures we would still create a dependency between the 2 that would block the admin client thread.  Perhaps the second future would have to already have a dependency on the users future built into it.
   
   One thing we have to be careful about is the constraints we put on the arguments to the constructor -- those constraints, whatever they end up being, have to be clearly documented.  If we don't have to impose any -- then that's great.  But if we have to impose/document some, then we should compare that with the 2-level results hierarchy I had provided earlier -- that imposes no constraints whatsoever.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r468871453



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,64 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * <p>Describe all SASL/SCRAM credentials.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials() {
+        return describeUserScramCredentials(null, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials for the given users.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null
+     *              or empty.  A user explicitly specified here that does not have a SCRAM credential will not appear
+     *              in the results.

Review comment:
       I originally was thinking we could just omit them.  But thinking about it more, I think we should have an error code to be consistent with our other APIs.  We'll need a new error code like USER_NOT_FOUND.  And we'll also need to add a per-user error code-- something we don't currently have.  But it shouldn't be too hard to do.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465367407



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
##########
@@ -1214,6 +1215,60 @@ default AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlterati
      */
     AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
 
+    /**
+     * <p>Describe all SASL/SCRAM credentials.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials() {
+        return describeUserScramCredentials(null, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials for the given users.
+     *
+     * <p>This is a convenience method for {@link #describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null or empty
+     * @return The DescribeUserScramCredentialsResult.
+     */
+    default DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users) {
+        return describeUserScramCredentials(users, new DescribeUserScramCredentialsOptions());
+    }
+
+    /**
+     * <p>Describe SASL/SCRAM credentials.
+     *
+     * @param users the users for which credentials are to be described; all users' credentials are described if null or empty

Review comment:
       It would be good to document here that users that are specified but not  present will be silently omitted from the result




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r478646950



##########
File path: tests/kafkatest/services/security/security_config.py
##########
@@ -350,6 +362,14 @@ def kafka_opts(self):
         else:
             return ""
 
+    def export_kafka_opts_for_admin_client_as_broker(self):

Review comment:
       As we discussed offline, I think eventually the admin client should log in as its own admin user, not as the broker.  But we can save that for a follow-on.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480272048



##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -575,10 +577,21 @@ def set_unclean_leader_election(self, topic, value=True, node=None):
         node.account.ssh(cmd)
 
     def _connect_setting_kafka_configs(self, node):
+        # Use this for everything related to kafka-configs except User SCRAM Credentials
         if node.version.kafka_configs_command_uses_bootstrap_server():
-            return "--bootstrap-server %s " % self.bootstrap_servers(self.security_protocol)
+            return "--bootstrap-server %s --command-config <(echo '%s')" % (self.bootstrap_servers(self.security_protocol),

Review comment:
       I created https://issues.apache.org/jira/browse/KAFKA-10451 to track this.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469627624



##########
File path: core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala
##########
@@ -0,0 +1,128 @@
+/*
+ * 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 kafka.server
+
+import java.util
+import java.util.Properties
+
+import kafka.network.SocketServer
+import kafka.security.authorizer.AclAuthorizer
+import org.apache.kafka.common.acl.AclOperation
+import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData
+import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
+import org.apache.kafka.common.resource.ResourceType
+import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, KafkaPrincipalBuilder}
+import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
+import org.junit.Assert._
+import org.junit.{Before, Test}
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Test DescribeUserScramCredentialsRequest/Response API for the cases where no credentials exist
+ * or failure is expected due to lack of authorization, sending the request to a non-controller broker, or some other issue.
+ * Testing the API for the case where there are actually credentials to describe is performed elsewhere.
+ */
+class DescribeUserScramCredentialsRequestTest extends BaseRequestTest {
+  override def brokerPropertyOverrides(properties: Properties): Unit = {
+    properties.put(KafkaConfig.ControlledShutdownEnableProp, "false")
+    properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[DescribeCredentialsTest.TestAuthorizer].getName)
+    properties.put(KafkaConfig.PrincipalBuilderClassProp, classOf[DescribeCredentialsTest.TestPrincipalBuilder].getName)
+  }
+
+  @Before
+  override def setUp(): Unit = {
+    DescribeCredentialsTest.principal = KafkaPrincipal.ANONYMOUS // default is to be authorized
+    super.setUp()
+  }
+
+  @Test
+  def testDescribeNothing(): Unit = {
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response = sendDescribeUserScramCredentialsRequest(request)
+
+    val error = response.data.error
+    assertEquals("Expected no error when routed correctly", Errors.NONE.code, error)
+    assertEquals("Expected no credentials", 0, response.data.userScramCredentials.size)
+  }
+
+  @Test
+  def testDescribeNotController(): Unit = {
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response = sendDescribeUserScramCredentialsRequest(request, notControllerSocketServer)
+
+    val error = response.data.error
+    assertEquals("Expected controller error when routed incorrectly", Errors.NOT_CONTROLLER.code, error)
+  }
+
+  @Test
+  def testDescribeNotAuthorized(): Unit = {
+    DescribeCredentialsTest.principal = DescribeCredentialsTest.UnauthorizedPrincipal
+
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData()).build()
+    val response = sendDescribeUserScramCredentialsRequest(request)
+
+    val error = response.data.error
+    assertEquals("Expected not authorized error", Errors.CLUSTER_AUTHORIZATION_FAILED.code, error)
+  }
+
+  @Test
+  def testDescribeSameUserTwice(): Unit = {
+    val user = new UserName().setName("user1")
+    val request = new DescribeUserScramCredentialsRequest.Builder(
+      new DescribeUserScramCredentialsRequestData().setUsers(List(user, user).asJava)).build()
+    val response = sendDescribeUserScramCredentialsRequest(request)
+
+    val error = response.data.error
+    assertEquals("Expected invalid request error", Errors.INVALID_REQUEST.code, error)
+  }
+
+
+  private def sendDescribeUserScramCredentialsRequest(request: DescribeUserScramCredentialsRequest, socketServer: SocketServer = controllerSocketServer): DescribeUserScramCredentialsResponse = {
+    connectAndReceive[DescribeUserScramCredentialsResponse](request, destination = socketServer)
+  }
+}
+
+object DescribeCredentialsTest {
+  val UnauthorizedPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "Unauthorized")

Review comment:
       @cmccabe Thanks.  I copied this approach from `kafka.server.RequestQuotaTest`.  I fixed it here in `DesrcibeUserScramCredentialsRequestTest` and `AlterUserScramCredentialsRequestTest` but was unable to fix it in `RequestQuotaTest` because that test is checking every possible API, and of course certain APIs need to succeed for Kafka to start successfully.  So that test needs to mutate the behavior after `setUp()`.  These tests don't since these APIs aren't necessary for Kafka to start.  So I was able to fix it here but was unable to fix it where I got the approach from.




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r472575326



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<Map<String, UserScramCredentialsDescription>> future;

Review comment:
       I can see reasonable arguments for all the possibilities, so if you feel strongly about one of them then I would be fine with it.  For example, `list()` and `describe()` even though they return the same thing (now -- `describe()` could return more later potentially).  Just let me know.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r478652254



##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -575,10 +577,21 @@ def set_unclean_leader_election(self, topic, value=True, node=None):
         node.account.ssh(cmd)
 
     def _connect_setting_kafka_configs(self, node):
+        # Use this for everything related to kafka-configs except User SCRAM Credentials
         if node.version.kafka_configs_command_uses_bootstrap_server():
-            return "--bootstrap-server %s " % self.bootstrap_servers(self.security_protocol)
+            return "--bootstrap-server %s --command-config <(echo '%s')" % (self.bootstrap_servers(self.security_protocol),

Review comment:
       It's sort of weird that we're creating files on the fly with `'<(echo '%s')` rather than just using a file in `/mnt/security`.  This will result in some pretty long command lines, right?
   
   Considering this is an existing pattern in `kafka.py` (but nowhere else?), let's file a follow-on JIRA to look into this and fix it.  Unless there's some really good reason why `kafka.py` is doing this, but I can't think of any.




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

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



[GitHub] [kafka] cmccabe commented on pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#issuecomment-687354300


   LGTM


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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r475695750



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<List<String>> usersFuture;
+    private final Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures;
+
+    /**
+     *
+     * @param usersFuture the future indicating the users described by the call
+     * @param perUserFutures the required map of user names to futures representing the results of describing each
+     *                       user's SCRAM credentials.
+     */
+    public DescribeUserScramCredentialsResult(KafkaFuture<List<String>> usersFuture,
+                                              Map<String, KafkaFuture<UserScramCredentialsDescription>> perUserFutures) {
+        this.usersFuture = Objects.requireNonNull(usersFuture);
+        this.perUserFutures = Objects.requireNonNull(perUserFutures);
+    }
+
+    /**
+     *
+     * @return a future for the results of all requested (either explicitly or implicitly via describe-all) users.
+     * The future will complete successfully only if the users future first completes successfully and then all the
+     * futures for the user descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(users());
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            KafkaFuture<Void> succeedsOnlyIfAllDescriptionsSucceed = KafkaFuture.allOf(perUserFutures.values().toArray(
+                    new KafkaFuture[perUserFutures.size()]));
+            KafkaFuture<Map<String, UserScramCredentialsDescription>> mapFuture = succeedsOnlyIfAllDescriptionsSucceed.thenApply(void2 ->
+                perUserFutures.entrySet().stream().collect(Collectors.toMap(
+                    e -> e.getKey(),
+                    e -> valueFromFutureGuaranteedToSucceedAtThisPoint(e.getValue()))));
+            /* At this point it is only the users future that is guaranteed to have succeeded.
+             * We want to return the future to the map, but we have to return a map at this point.
+             * We need to dereference the future while propagating any exception.
+             */
+            return valueFromFuturePropagatingExceptionsAsUnchecked(mapFuture);
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that were requested (either explicitly or implicitly via
+     * describe-all).  The future will not complete successfully if the user is not authorized to perform the describe
+     * operation; otherwise, it will complete successfully as long as the list of users with credentials can be
+     * successfully determined within some hard-coded timeout period.
+     */
+    public KafkaFuture<List<String>> users() {
+        return usersFuture;
+    }
+
+    /**
+     *
+     * @param userName the name of the user description being requested
+     * @return a future indicating the description results for the given user. The future will complete exceptionally if
+     * the future returned by {@link #users()} completes exceptionally.  If the given user does not exist in the list
+     * of requested users then the future will complete exceptionally with
+     * {@link org.apache.kafka.common.errors.ResourceNotFoundException}.
+     */
+    public KafkaFuture<UserScramCredentialsDescription> description(String userName) {
+        KafkaFuture<Void> succeedsOnlyIfUsersFutureSucceeds = KafkaFuture.allOf(usersFuture);
+        return succeedsOnlyIfUsersFutureSucceeds.thenApply(void1 -> {
+            // it is possible that there is no future for this user (for example, the original describe request was for
+            // users 1, 2, and 3 but this is looking for user 4), so explicitly take care of that case
+            KafkaFuture<UserScramCredentialsDescription> requestedUserFuture = perUserFutures.get(userName);
+            if (requestedUserFuture == null) {
+                throw new ResourceNotFoundException("No such user: " + userName);
+            }
+            KafkaFuture<Void> succeedsOnlyIfRequestedUserFutureSucceeds = KafkaFuture.allOf(requestedUserFuture);

Review comment:
       We don't need this extra future, right?  We can just use requestedUserFuture directly if we want to use that




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r480282139



##########
File path: core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala
##########
@@ -169,6 +169,18 @@ object JaasTestUtils {
     jaasFile
   }
 
+  // Returns a SASL/SCRAM configuration using credentials for the given user and password
+  def scramClientLoginModule(mechanism: String, scramUser: String, scramPassword: String): String = {
+    mechanism match {
+      case "SCRAM-SHA-256" | "SCRAM-SHA-512" =>

Review comment:
       I fixed this in the other places in this file as well




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

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



[GitHub] [kafka] rondagostino commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
rondagostino commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r469421523



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
##########
@@ -4071,6 +4081,168 @@ void handleFailure(Throwable throwable) {
         return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
     }
 
+    @Override
+    public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
+        final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> future = new KafkaFutureImpl<>();
+        final long now = time.milliseconds();
+        Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
+                new ControllerNodeProvider()) {
+            @Override
+            public DescribeUserScramCredentialsRequest.Builder createRequest(int timeoutMs) {
+                return new DescribeUserScramCredentialsRequest.Builder(
+                        new DescribeUserScramCredentialsRequestData().setUsers(users.stream().map(user ->
+                                new DescribeUserScramCredentialsRequestData.UserName().setName(user)).collect(Collectors.toList())));
+            }
+
+            @Override
+            public void handleResponse(AbstractResponse abstractResponse) {
+                DescribeUserScramCredentialsResponse response = (DescribeUserScramCredentialsResponse) abstractResponse;
+                Errors error = Errors.forCode(response.data().error());
+                switch (error) {
+                    case NONE:
+                        DescribeUserScramCredentialsResponseData data = response.data();
+                        future.complete(data.userScramCredentials().stream().collect(Collectors.toMap(
+                            DescribeUserScramCredentialsResponseData.UserScramCredential::name,
+                            userScramCredential -> {
+                                List<ScramCredentialInfo> scramCredentialInfos = userScramCredential.credentialInfos().stream().map(
+                                    credentialInfo -> new ScramCredentialInfo(ScramMechanism.from(credentialInfo.mechanism()), credentialInfo.iterations()))
+                                        .collect(Collectors.toList());
+                                return new UserScramCredentialsDescription(userScramCredential.name(), scramCredentialInfos);
+                            })));
+                        break;
+                    case NOT_CONTROLLER:
+                        handleNotControllerError(error);
+                        break;
+                    default:
+                        future.completeExceptionally(new ApiError(error, response.data().errorMessage()).exception());
+                        break;
+                }
+            }
+
+            @Override
+            void handleFailure(Throwable throwable) {
+                future.completeExceptionally(throwable);
+            }
+        };
+        runnable.call(call, now);
+        return new DescribeUserScramCredentialsResult(future);
+    }
+
+    @Override
+    public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
+                                                                     AlterUserScramCredentialsOptions options) {
+        final long now = time.milliseconds();
+        final Map<String, KafkaFutureImpl<Void>> futures = new HashMap<>();
+        for (UserScramCredentialAlteration alteration: alterations) {
+            futures.put(alteration.getUser(), new KafkaFutureImpl<>());
+        }
+        final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
+        // We need to keep track of users with deletions of an unknown SCRAM mechanism
+        alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
+            UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
+            ScramMechanism mechanism = deletion.getMechanism();
+            if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
+                userIllegalAlterationExceptions.put(deletion.getUser(), new IllegalArgumentException("Unknown SCRAM mechanism"));
+            }
+        });
+        // Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
+        // so keep track of which users are affected by such a failure and immediately fail all their alterations
+        final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
+        alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
+                .filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.getUser()))
+                .forEach(alteration -> {
+                    UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
+                    String user = upsertion.getUser();
+                    try {
+                        ScramMechanism mechanism = upsertion.getInfo().getMechanism();
+                        if (mechanism == null || mechanism == ScramMechanism.UNKNOWN)
+                            throw new IllegalArgumentException("Unknown SCRAM mechanism");
+                        userInsertions.putIfAbsent(user, new HashMap<>());
+                        userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
+                    } catch (Exception e) {
+                        // we might overwrite aa exception from a previous upsertion, but we don't really care
+                        // since we just needs to mark this user as having at least one illegal alteration
+                        // and make an exception instance available for completing the corresponding future exceptionally
+                        userIllegalAlterationExceptions.put(user, e);

Review comment:
       `NoSuchAlgorithmException` is thrown if we don't support the (otherwise valid) SCRAM mechanism.  For example, this might occur if we add support for a new SCRAM mechanism but the client is running an older JRE that doesn't support the algorithm associated with that newer mechanism.  `UnsupportedSaslMechanismException` would be appropriate here, I think.
   
   `InvalidKeyException` is thrown if the password to be salted is unacceptable -- i.e. if it is empty.  I will add a new UnacceptablePasswordException and Errors.UNACCEPTABLE_PASSWORD as a starting point for further review/discussion.




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r479546505



##########
File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.common.KafkaFuture;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.ResourceNotFoundException;
+import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
+import org.apache.kafka.common.protocol.Errors;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+/**
+ * The result of the {@link Admin#describeUserScramCredentials()} call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeUserScramCredentialsResult {
+    private final KafkaFuture<DescribeUserScramCredentialsResponseData> dataFuture;
+
+    /**
+     * Package-private constructor
+     *
+     * @param dataFuture the future indicating response data from the call
+     */
+    DescribeUserScramCredentialsResult(KafkaFuture<DescribeUserScramCredentialsResponseData> dataFuture) {
+        this.dataFuture = Objects.requireNonNull(dataFuture);
+    }
+
+    /**
+     *
+     * @return a future for the results of all described users with map keys (one per user) being consistent with the
+     * contents of the list returned by {@link #users()}. The future will complete successfully only if all such user
+     * descriptions complete successfully.
+     */
+    public KafkaFuture<Map<String, UserScramCredentialsDescription>> all() {
+        return KafkaFuture.allOf(dataFuture).thenApply(v -> {
+            DescribeUserScramCredentialsResponseData data = valueFromFutureGuaranteedToSucceedAtThisPoint(dataFuture);
+            /* Check to make sure every individual described user succeeded.  Note that a successfully described user
+             * is one that appears with *either* a NONE error code or a RESOURCE_NOT_FOUND error code. The
+             * RESOURCE_NOT_FOUND means the client explicitly requested a describe of that particular user but it could
+             * not be described because it does not exist; such a user will not appear as a key in the returned map.
+             */
+            Optional<DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult> optionalFirstFailedDescribe =
+                    data.results().stream().filter(result ->
+                        result.errorCode() != Errors.NONE.code() && result.errorCode() != Errors.RESOURCE_NOT_FOUND.code()).findFirst();
+            if (optionalFirstFailedDescribe.isPresent()) {
+                throw Errors.forCode(optionalFirstFailedDescribe.get().errorCode()).exception(optionalFirstFailedDescribe.get().errorMessage());
+            }
+            Map<String, UserScramCredentialsDescription> retval = new HashMap<>();
+            data.results().stream().forEach(userResult ->
+                    retval.put(userResult.user(), new UserScramCredentialsDescription(userResult.user(),
+                            getScramCredentialInfosFor(userResult))));
+            return retval;
+        });
+    }
+
+    /**
+     *
+     * @return a future indicating the distinct users that meet the request criteria and that have at least one
+     * credential.  The future will not complete successfully if the user is not authorized to perform the describe
+     * operation; otherwise, it will complete successfully as long as the list of users with credentials can be
+     * successfully determined within some hard-coded timeout period. Note that the returned list will not include users
+     * that do not exist/have no credentials: a request to describe an explicit list of users, none of which existed/had
+     * a credential, will result in a future that returns an empty list being returned here. A returned list will
+     * include users that have a credential but that could not be described.
+     */
+    public KafkaFuture<List<String>> users() {
+        return KafkaFuture.allOf(dataFuture).thenApply(v -> {
+            DescribeUserScramCredentialsResponseData data = valueFromFutureGuaranteedToSucceedAtThisPoint(dataFuture);
+            return data.results().stream()
+                    .filter(result -> result.errorCode() != Errors.RESOURCE_NOT_FOUND.code())
+                    .map(result -> result.user()).collect(Collectors.toList());
+        });
+    }
+
+    /**
+     *
+     * @param userName the name of the user description being requested
+     * @return a future indicating the description results for the given user. The future will complete exceptionally if
+     * the future returned by {@link #users()} completes exceptionally.  Note that if the given user does not exist in
+     * the list of described users then the returned future will complete exceptionally with
+     * {@link org.apache.kafka.common.errors.ResourceNotFoundException}.
+     */
+    public KafkaFuture<UserScramCredentialsDescription> description(String userName) {
+        return KafkaFuture.allOf(dataFuture).thenApply(v -> {
+            DescribeUserScramCredentialsResponseData data = valueFromFutureGuaranteedToSucceedAtThisPoint(dataFuture);
+            // it is possible that there is no future for this user (for example, the original describe request was for
+            // users 1, 2, and 3 but this is looking for user 4), so explicitly take care of that case
+            Optional<DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult> optionalUserResult =
+                    data.results().stream().filter(result -> result.user().equals(userName)).findFirst();
+            if (!optionalUserResult.isPresent()) {
+                throw new ResourceNotFoundException("No such user: " + userName);
+            }
+            DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult userResult = optionalUserResult.get();
+            if (userResult.errorCode() != Errors.NONE.code()) {
+                // RESOURCE_NOT_FOUND is included here
+                throw Errors.forCode(userResult.errorCode()).exception(userResult.errorMessage());
+            }
+            return new UserScramCredentialsDescription(userResult.user(), getScramCredentialInfosFor(userResult));
+        });
+    }
+
+    private static List<ScramCredentialInfo> getScramCredentialInfosFor(
+            DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult userResult) {
+        return userResult.credentialInfos().stream().map(c ->
+                new ScramCredentialInfo(ScramMechanism.fromType(c.mechanism()), c.iterations()))
+                .collect(Collectors.toList());
+    }
+
+    private static <T> T valueFromFutureGuaranteedToSucceedAtThisPoint(KafkaFuture<T> future) {
+        try {
+            return future.get();

Review comment:
       The call to get here isn't needed.  See the comment about about using `KafkaFuture#whenComplete`.  Chaining future results is a big part of the power of `CompletableFuture` (which `KafkaFuture` is a clone of... long story...)




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r465393358



##########
File path: core/src/main/scala/kafka/server/AdminManager.scala
##########
@@ -980,4 +984,137 @@ class AdminManager(val config: KafkaConfig,
       entry.entity -> apiError
     }.toMap
   }
+
+  def describeUserScramCredentials(users: Seq[String]): DescribeUserScramCredentialsResponseData = {
+    val retval = new DescribeUserScramCredentialsResponseData()
+
+    def addToResults(user: String, userConfig: Properties) = {
+      val configKeys = userConfig.stringPropertyNames
+      val hasScramCredential = !ScramMechanism.values().toList.filter(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.toMechanismName)).isEmpty
+      if (hasScramCredential) {
+        val userScramCredentials = new UserScramCredential().setName(user)
+        ScramMechanism.values().foreach(mechanism => if (mechanism != ScramMechanism.UNKNOWN) {
+          val propertyValue = userConfig.getProperty(mechanism.toMechanismName)
+          if (propertyValue != null) {
+            val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
+            userScramCredentials.credentialInfos.add(new CredentialInfo().setMechanism(mechanism.ordinal.toByte).setIterations(iterations))
+          }
+        })
+        retval.userScramCredentials.add(userScramCredentials)
+      }
+    }
+
+    if (users.isEmpty)

Review comment:
       It would be good to avoid confusing empty with null.  Why not use Option[Seq[String]]




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

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



[GitHub] [kafka] cmccabe commented on a change in pull request #9032: KAFKA-10259: KIP-554 Broker-side SCRAM Config API

Posted by GitBox <gi...@apache.org>.
cmccabe commented on a change in pull request #9032:
URL: https://github.com/apache/kafka/pull/9032#discussion_r478652254



##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -575,10 +577,21 @@ def set_unclean_leader_election(self, topic, value=True, node=None):
         node.account.ssh(cmd)
 
     def _connect_setting_kafka_configs(self, node):
+        # Use this for everything related to kafka-configs except User SCRAM Credentials
         if node.version.kafka_configs_command_uses_bootstrap_server():
-            return "--bootstrap-server %s " % self.bootstrap_servers(self.security_protocol)
+            return "--bootstrap-server %s --command-config <(echo '%s')" % (self.bootstrap_servers(self.security_protocol),

Review comment:
       It's sort of weird that we're creating files on the fly with '<(echo '%s')` rather than just using a file in /mnt/security.  This will result in some pretty long command lines, right?
   
   Considering this is an existing pattern in kafka.py (but nowhere else?), let's file a follow-on JIRA to look into this and fix it.  Unless there's some really good reason why `kafka.py` is doing this, but I can't think of any.




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

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