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

2023-02-16 Thread via GitHub


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


##
core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala:
##
@@ -17,18 +17,22 @@
 package kafka.server
 
 import java.util
-import java.util.Properties
+// import java.util.Properties

Review Comment:
   can we remove unused import?



##
core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala:
##
@@ -260,11 +271,13 @@ class AlterUserScramCredentialsRequestTest extends 
BaseRequestTest {
 .setSalt(saltBytes)
 .setSaltedPassword(saltedPasswordBytes),
 ))).build()
-val results1 = sendAlterUserScramCredentialsRequest(request1).data.results
-assertEquals(2, results1.size)
-checkNoErrorsAlteringCredentials(results1)
-checkUserAppearsInAlterResults(results1, user1)
-checkUserAppearsInAlterResults(results1, user2)
+val results1_1 = 
sendAlterUserScramCredentialsRequest(request1_1).data.results
+assertEquals(2, results1_1.size)
+checkNoErrorsAlteringCredentials(results1_1)
+checkUserAppearsInAlterResults(results1_1, user1)
+checkUserAppearsInAlterResults(results1_1, user2)
+
+Thread.sleep(1)

Review Comment:
   Can we address 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.

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

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



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

2023-02-15 Thread via GitHub


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


##
core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala:
##
@@ -260,11 +271,13 @@ class AlterUserScramCredentialsRequestTest extends 
BaseRequestTest {
 .setSalt(saltBytes)
 .setSaltedPassword(saltedPasswordBytes),
 ))).build()
-val results1 = sendAlterUserScramCredentialsRequest(request1).data.results
-assertEquals(2, results1.size)
-checkNoErrorsAlteringCredentials(results1)
-checkUserAppearsInAlterResults(results1, user1)
-checkUserAppearsInAlterResults(results1, user2)
+val results1_1 = 
sendAlterUserScramCredentialsRequest(request1_1).data.results
+assertEquals(2, results1_1.size)
+checkNoErrorsAlteringCredentials(results1_1)
+checkUserAppearsInAlterResults(results1_1, user1)
+checkUserAppearsInAlterResults(results1_1, user2)
+
+Thread.sleep(1)

Review Comment:
   we generally use 'TestUtils.waitForCondition` in tests to wait for condition



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

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

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



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

2023-02-15 Thread via GitHub


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


##
core/src/main/scala/kafka/server/KafkaApis.scala:
##
@@ -3296,17 +3296,23 @@ class KafkaApis(val requestChannel: RequestChannel,
   }
 
   def handleDescribeUserScramCredentialsRequest(request: 
RequestChannel.Request): Unit = {

Review Comment:
   As I mentioned above,  ControllerAPI also should support 
DescribeUserScramCredentialsRequest API



##
core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala:
##
@@ -260,11 +271,13 @@ class AlterUserScramCredentialsRequestTest extends 
BaseRequestTest {
 .setSalt(saltBytes)
 .setSaltedPassword(saltedPasswordBytes),
 ))).build()
-val results1 = sendAlterUserScramCredentialsRequest(request1).data.results
-assertEquals(2, results1.size)
-checkNoErrorsAlteringCredentials(results1)
-checkUserAppearsInAlterResults(results1, user1)
-checkUserAppearsInAlterResults(results1, user2)
+val results1_1 = 
sendAlterUserScramCredentialsRequest(request1_1).data.results
+assertEquals(2, results1_1.size)
+checkNoErrorsAlteringCredentials(results1_1)
+checkUserAppearsInAlterResults(results1_1, user1)
+checkUserAppearsInAlterResults(results1_1, user2)
+
+Thread.sleep(1)

Review Comment:
   we generally use 'TestUtils.waitForCondition` in tests 



##
metadata/src/main/java/org/apache/kafka/image/ScramImage.java:
##
@@ -50,6 +58,63 @@ public void write(ImageWriter writer, ImageWriterOptions 
options) {
 }
 }
 
+public DescribeUserScramCredentialsResponseData 
describe(DescribeUserScramCredentialsRequestData request) {
+
+List users = request.users();
+Map uniqueUsers = new HashMap();
+
+if ((users == null) || (users.size() == 0)) {
+System.out.println("Describe : get all the users");
+// If there are no users listed then get all the users
+for (Map scramCredentialDataSet : 
mechanisms.values()) {
+for (String user : scramCredentialDataSet.keySet()) {
+uniqueUsers.put(user, false);
+}
+}
+} else {
+// Filter out duplicates
+for (UserName user : users) {
+if (uniqueUsers.containsKey(user.name())) {
+uniqueUsers.put(user.name(), true);
+} else {
+uniqueUsers.put(user.name(), false);
+}
+}
+}
+
+DescribeUserScramCredentialsResponseData retval = new 
DescribeUserScramCredentialsResponseData();
+
+for (Map.Entry user : uniqueUsers.entrySet()) {
+DescribeUserScramCredentialsResult result = 
+  new DescribeUserScramCredentialsResult().setUser(user.getKey());
+
+if (user.getValue() == false) {
+List credentialInfos = new 
ArrayList();
+
+boolean datafound = false;
+for (Map.Entry> mechanismsEntry : mechanisms.entrySet()) {
+Map credentialDataSet = 
mechanismsEntry.getValue();
+if (credentialDataSet.containsKey(user.getKey())) {
+credentialInfos.add(new 
CredentialInfo().setMechanism(mechanismsEntry.getKey().type())
+
.setIterations(credentialDataSet.get(user.getKey()).iterations()));
+datafound = true;
+}
+}
+if (datafound) {
+result.setCredentialInfos(credentialInfos);
+} else {
+result.setErrorCode(Errors.RESOURCE_NOT_FOUND.code())
+  
.setErrorMessage("attemptToDescribeUserThatDoesNotExist: " + user.getKey());

Review Comment:
   Can we use same error message as 
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ZkAdminManager.scala#L802



##
metadata/src/main/java/org/apache/kafka/image/ScramImage.java:
##
@@ -50,6 +58,63 @@ public void write(ImageWriter writer, ImageWriterOptions 
options) {
 }
 }
 
+public DescribeUserScramCredentialsResponseData 
describe(DescribeUserScramCredentialsRequestData request) {
+
+List users = request.users();
+Map uniqueUsers = new HashMap();
+
+if ((users == null) || (users.size() == 0)) {
+System.out.println("Describe : get all the users");
+// If there are no users listed then get all the users
+for (Map scramCredentialDataSet : 
mechanisms.values()) {
+for (String user : scramCredentialDataSet.keySet()) {
+uniqueUsers.put(user, false);
+}
+}
+} else {
+// Filter out duplicates
+for (UserName user : users) {
+  

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

2023-02-09 Thread via GitHub


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


##
metadata/src/main/java/org/apache/kafka/controller/ScramControlManager.java:
##
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.ScramMechanism;
+import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
+import 
org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialDeletion;
+import 
org.apache.kafka.common.message.AlterUserScramCredentialsRequestData.ScramCredentialUpsertion;
+import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
+import 
org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult;
+import org.apache.kafka.common.metadata.RemoveUserScramCredentialRecord;
+import org.apache.kafka.common.metadata.UserScramCredentialRecord;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import static org.apache.kafka.common.protocol.Errors.DUPLICATE_RESOURCE;
+import static org.apache.kafka.common.protocol.Errors.NONE;
+import static org.apache.kafka.common.protocol.Errors.RESOURCE_NOT_FOUND;
+import static org.apache.kafka.common.protocol.Errors.UNACCEPTABLE_CREDENTIAL;
+import static 
org.apache.kafka.common.protocol.Errors.UNSUPPORTED_SASL_MECHANISM;
+
+
+/**
+ * Manages SCRAM credentials.
+ */
+public class ScramControlManager {
+static final int MAX_ITERATIONS = 16384;
+
+static class Builder {
+private LogContext logContext = null;
+private SnapshotRegistry snapshotRegistry = null;
+
+Builder setLogContext(LogContext logContext) {
+this.logContext = logContext;
+return this;
+}
+
+Builder setSnapshotRegistry(SnapshotRegistry snapshotRegistry) {
+this.snapshotRegistry = snapshotRegistry;
+return this;
+}
+
+ScramControlManager build() {
+if (logContext == null) logContext = new LogContext();
+if (snapshotRegistry == null) snapshotRegistry = new 
SnapshotRegistry(logContext);
+return new ScramControlManager(logContext,
+snapshotRegistry);
+}
+}
+
+static class ScramCredentialKey {
+private final String username;
+private final ScramMechanism mechanism;
+
+ScramCredentialKey(String username, ScramMechanism mechanism) {
+this.username = username;
+this.mechanism = mechanism;
+}
+
+@Override
+public int hashCode() {
+return Objects.hash(username, mechanism);
+}
+
+@Override
+public boolean equals(Object o) {
+if (o == null) return false;
+if (!(o.getClass() == this.getClass())) return false;
+ScramCredentialKey other = (ScramCredentialKey) o;
+return username.equals(other.username) &&
+mechanism.equals(other.mechanism);
+}
+
+@Override
+public String toString() {
+return "ScramCredentialKey" +
+"(username=" + username +
+", mechanism=" + mechanism +
+")";
+}
+}
+
+static class ScramCredentialValue {
+private final byte[] salt;
+private final byte[] saltedPassword;
+private final int iterations;
+
+ScramCredentialValue(
+byte[] salt,
+byte[] saltedPassword,
+int iterations
+) {
+this.salt = salt;
+this.saltedPassword = saltedPassword;
+this.iterations = iterations;
+}
+
+@Override
+

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

2023-02-08 Thread via GitHub


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


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

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



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

Review Comment:
   unwanted line



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

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



##
metadata/src/main/java/org/apache/kafka/image/ScramCredentialData.java:
##
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.image;
+
+import org.apache.kafka.clients.admin.ScramMechanism;
+import org.apache.kafka.common.metadata.UserScramCredentialRecord;
+import org.apache.kafka.common.security.scram.ScramCredential;
+import org.apache.kafka.common.security.scram.internals.ScramFormatter;
+import org.apache.kafka.common.utils.Bytes;
+
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import java.util.Objects;
+
+
+/**
+ * Represents the ACLs in the metadata image.
+ *
+ * This class is thread-safe.
+ */
+public final class ScramCredentialData {
+private final byte[] salt;
+private final byte[] saltedPassword;
+private final int iterations;
+
+static ScramCredentialData fromRecord(
+UserScramCredentialRecord record
+) {
+return new ScramCredentialData(
+record.salt(),
+record.saltedPassword(),
+record.iterations());
+}
+
+public ScramCredentialData(
+byte[] salt,
+byte[] saltedPassword,
+int iterations
+) {
+this.salt = salt;
+this.saltedPassword = saltedPassword;
+this.iterations = iterations;
+}
+
+public byte[] salt() {
+return salt;
+}
+
+public byte[] saltedPassword() {
+return saltedPassword;
+}
+
+public int iterations() {
+return iterations;
+}
+
+public UserScramCredentialRecord toRecord(
+String userName,
+ScramMechanism mechanism
+) {
+return new UserScramCredentialRecord().
+setName(userName).
+setMechanism(mechanism.type()).
+setSalt(salt).
+