mirror of https://github.com/apache/kafka.git
KAFKA-10259: KIP-554 Broker-side SCRAM Config API (#9032)
Implement the KIP-554 API to create, describe, and alter SCRAM user configurations via the AdminClient. Add ducktape tests, and modify JUnit tests to test and use the new API where appropriate. Reviewers: Colin P. McCabe <cmccabe@apache.org>, Rajini Sivaram <rajinisivaram@googlemail.com>
This commit is contained in:
parent
c6d135e21f
commit
e8524ccd8f
|
@ -20,6 +20,7 @@ package org.apache.kafka.clients.admin;
|
|||
import java.time.Duration;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
@ -1214,6 +1215,97 @@ public interface Admin extends AutoCloseable {
|
|||
*/
|
||||
AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options);
|
||||
|
||||
/**
|
||||
* 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());
|
||||
}
|
||||
|
||||
/**
|
||||
* 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());
|
||||
}
|
||||
|
||||
/**
|
||||
* Describe SASL/SCRAM credentials.
|
||||
* <p>
|
||||
* The following exceptions can be anticipated when calling {@code get()} on the futures from the
|
||||
* returned {@link DescribeUserScramCredentialsResult}:
|
||||
* <ul>
|
||||
* <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
|
||||
* If the authenticated user didn't have describe access to the cluster.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.ResourceNotFoundException}
|
||||
* If the user did not exist/had no SCRAM credentials.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.DuplicateResourceException}
|
||||
* If the user was requested to be described more than once in the original request.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.TimeoutException}
|
||||
* If the request timed out before the describe operation could finish.</li>
|
||||
* </ul>
|
||||
* <p>
|
||||
* This operation is supported by brokers with version 2.7.0 or higher.
|
||||
*
|
||||
* @param users the users for which credentials are to be described; all users' credentials are described if null
|
||||
* or empty.
|
||||
* @param options The options to use when describing the credentials
|
||||
* @return The DescribeUserScramCredentialsResult.
|
||||
*/
|
||||
DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options);
|
||||
|
||||
/**
|
||||
* Alter SASL/SCRAM credentials for the given users.
|
||||
*
|
||||
* <p>This is a convenience method for {@link #alterUserScramCredentials(List, AlterUserScramCredentialsOptions)}
|
||||
*
|
||||
* @param alterations the alterations to be applied
|
||||
* @return The AlterUserScramCredentialsResult.
|
||||
*/
|
||||
default AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations) {
|
||||
return alterUserScramCredentials(alterations, new AlterUserScramCredentialsOptions());
|
||||
}
|
||||
|
||||
/**
|
||||
* Alter SASL/SCRAM credentials.
|
||||
*
|
||||
* <p>
|
||||
* The following exceptions can be anticipated when calling {@code get()} any of the futures from the
|
||||
* returned {@link AlterUserScramCredentialsResult}:
|
||||
* <ul>
|
||||
* <li>{@link org.apache.kafka.common.errors.NotControllerException}
|
||||
* If the request is not sent to the Controller broker.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.ClusterAuthorizationException}
|
||||
* If the authenticated user didn't have alter access to the cluster.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.UnsupportedByAuthenticationException}
|
||||
* If the user authenticated with a delegation token.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.UnsupportedSaslMechanismException}
|
||||
* If the requested SCRAM mechanism is unrecognized or otherwise unsupported.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.UnacceptableCredentialException}
|
||||
* If the username is empty or the requested number of iterations is too small or too large.</li>
|
||||
* <li>{@link org.apache.kafka.common.errors.TimeoutException}
|
||||
* If the request timed out before the describe could finish.</li>
|
||||
* </ul>
|
||||
* <p>
|
||||
* This operation is supported by brokers with version 2.7.0 or higher.
|
||||
*
|
||||
* @param alterations the alterations to be applied
|
||||
* @param options The options to use when altering the credentials
|
||||
* @return The AlterUserScramCredentialsResult.
|
||||
*/
|
||||
AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations,
|
||||
AlterUserScramCredentialsOptions options);
|
||||
|
||||
/**
|
||||
* Get the metrics kept by the adminClient
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* 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.annotation.InterfaceStability;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Options for {@link AdminClient#alterUserScramCredentials(List, AlterUserScramCredentialsOptions)}
|
||||
*
|
||||
* The API of this class is evolving. See {@link AdminClient} for details.
|
||||
*/
|
||||
@InterfaceStability.Evolving
|
||||
public class AlterUserScramCredentialsOptions extends AbstractOptions<AlterUserScramCredentialsOptions> {
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* 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.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* The result of the {@link Admin#alterUserScramCredentials(List)} call.
|
||||
*
|
||||
* The API of this class is evolving, see {@link Admin} for details.
|
||||
*/
|
||||
@InterfaceStability.Evolving
|
||||
public class AlterUserScramCredentialsResult {
|
||||
private final Map<String, KafkaFuture<Void>> futures;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param futures the required map from user names to futures representing the results of the alteration(s)
|
||||
* for each user
|
||||
*/
|
||||
public AlterUserScramCredentialsResult(Map<String, KafkaFuture<Void>> futures) {
|
||||
this.futures = Collections.unmodifiableMap(Objects.requireNonNull(futures));
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a map from user names to futures, which can be used to check the status of the alteration(s)
|
||||
* for each user.
|
||||
*/
|
||||
public Map<String, KafkaFuture<Void>> values() {
|
||||
return this.futures;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a future which succeeds only if all the user SCRAM credential alterations succeed.
|
||||
*/
|
||||
public KafkaFuture<Void> all() {
|
||||
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* 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.annotation.InterfaceStability;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Options for {@link AdminClient#describeUserScramCredentials(List, DescribeUserScramCredentialsOptions)}
|
||||
*
|
||||
* The API of this class is evolving. See {@link AdminClient} for details.
|
||||
*/
|
||||
@InterfaceStability.Evolving
|
||||
public class DescribeUserScramCredentialsOptions extends AbstractOptions<DescribeUserScramCredentialsOptions> {
|
||||
}
|
|
@ -0,0 +1,150 @@
|
|||
/*
|
||||
* 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.internals.KafkaFutureImpl;
|
||||
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.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() {
|
||||
final KafkaFutureImpl<Map<String, UserScramCredentialsDescription>> retval = new KafkaFutureImpl<>();
|
||||
dataFuture.whenComplete((data, throwable) -> {
|
||||
if (throwable != null) {
|
||||
retval.completeExceptionally(throwable);
|
||||
} else {
|
||||
/* 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()) {
|
||||
retval.completeExceptionally(Errors.forCode(optionalFirstFailedDescribe.get().errorCode()).exception(optionalFirstFailedDescribe.get().errorMessage()));
|
||||
} else {
|
||||
Map<String, UserScramCredentialsDescription> retvalMap = new HashMap<>();
|
||||
data.results().stream().forEach(userResult ->
|
||||
retvalMap.put(userResult.user(), new UserScramCredentialsDescription(userResult.user(),
|
||||
getScramCredentialInfosFor(userResult))));
|
||||
retval.complete(retvalMap);
|
||||
}
|
||||
}
|
||||
});
|
||||
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() {
|
||||
final KafkaFutureImpl<List<String>> retval = new KafkaFutureImpl<>();
|
||||
dataFuture.whenComplete((data, throwable) -> {
|
||||
if (throwable != null) {
|
||||
retval.completeExceptionally(throwable);
|
||||
} else {
|
||||
retval.complete(data.results().stream()
|
||||
.filter(result -> result.errorCode() != Errors.RESOURCE_NOT_FOUND.code())
|
||||
.map(result -> result.user()).collect(Collectors.toList()));
|
||||
}
|
||||
});
|
||||
return retval;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @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) {
|
||||
final KafkaFutureImpl<UserScramCredentialsDescription> retval = new KafkaFutureImpl<>();
|
||||
dataFuture.whenComplete((data, throwable) -> {
|
||||
if (throwable != null) {
|
||||
retval.completeExceptionally(throwable);
|
||||
} else {
|
||||
// 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()) {
|
||||
retval.completeExceptionally(new ResourceNotFoundException("No such user: " + userName));
|
||||
} else {
|
||||
DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult userResult = optionalUserResult.get();
|
||||
if (userResult.errorCode() != Errors.NONE.code()) {
|
||||
// RESOURCE_NOT_FOUND is included here
|
||||
retval.completeExceptionally(Errors.forCode(userResult.errorCode()).exception(userResult.errorMessage()));
|
||||
} else {
|
||||
retval.complete(new UserScramCredentialsDescription(userResult.user(), getScramCredentialInfosFor(userResult)));
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
return retval;
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
}
|
|
@ -65,8 +65,10 @@ import org.apache.kafka.common.errors.KafkaStorageException;
|
|||
import org.apache.kafka.common.errors.RetriableException;
|
||||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
|
||||
import org.apache.kafka.common.errors.TimeoutException;
|
||||
import org.apache.kafka.common.errors.UnacceptableCredentialException;
|
||||
import org.apache.kafka.common.errors.UnknownServerException;
|
||||
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
|
||||
import org.apache.kafka.common.errors.UnsupportedSaslMechanismException;
|
||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
||||
import org.apache.kafka.common.internals.KafkaFutureImpl;
|
||||
import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData;
|
||||
|
@ -76,6 +78,7 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData.AlterRepli
|
|||
import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData.AlterReplicaLogDirTopic;
|
||||
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirPartitionResult;
|
||||
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult;
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
|
||||
import org.apache.kafka.common.message.CreateAclsRequestData;
|
||||
import org.apache.kafka.common.message.CreateAclsRequestData.AclCreation;
|
||||
import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult;
|
||||
|
@ -112,6 +115,8 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup
|
|||
import org.apache.kafka.common.message.DescribeLogDirsRequestData;
|
||||
import org.apache.kafka.common.message.DescribeLogDirsRequestData.DescribableLogDirTopic;
|
||||
import org.apache.kafka.common.message.DescribeLogDirsResponseData;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.message.ExpireDelegationTokenRequestData;
|
||||
import org.apache.kafka.common.message.FindCoordinatorRequestData;
|
||||
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData;
|
||||
|
@ -157,6 +162,8 @@ import org.apache.kafka.common.requests.AlterPartitionReassignmentsRequest;
|
|||
import org.apache.kafka.common.requests.AlterPartitionReassignmentsResponse;
|
||||
import org.apache.kafka.common.requests.AlterReplicaLogDirsRequest;
|
||||
import org.apache.kafka.common.requests.AlterReplicaLogDirsResponse;
|
||||
import org.apache.kafka.common.requests.AlterUserScramCredentialsRequest;
|
||||
import org.apache.kafka.common.requests.AlterUserScramCredentialsResponse;
|
||||
import org.apache.kafka.common.requests.ApiError;
|
||||
import org.apache.kafka.common.requests.CreateAclsRequest;
|
||||
import org.apache.kafka.common.requests.CreateAclsResponse;
|
||||
|
@ -186,6 +193,8 @@ import org.apache.kafka.common.requests.DescribeGroupsRequest;
|
|||
import org.apache.kafka.common.requests.DescribeGroupsResponse;
|
||||
import org.apache.kafka.common.requests.DescribeLogDirsRequest;
|
||||
import org.apache.kafka.common.requests.DescribeLogDirsResponse;
|
||||
import org.apache.kafka.common.requests.DescribeUserScramCredentialsRequest;
|
||||
import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse;
|
||||
import org.apache.kafka.common.requests.ElectLeadersRequest;
|
||||
import org.apache.kafka.common.requests.ElectLeadersResponse;
|
||||
import org.apache.kafka.common.requests.ExpireDelegationTokenRequest;
|
||||
|
@ -215,6 +224,7 @@ import org.apache.kafka.common.requests.OffsetFetchResponse;
|
|||
import org.apache.kafka.common.requests.RenewDelegationTokenRequest;
|
||||
import org.apache.kafka.common.requests.RenewDelegationTokenResponse;
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramFormatter;
|
||||
import org.apache.kafka.common.security.token.delegation.DelegationToken;
|
||||
import org.apache.kafka.common.security.token.delegation.TokenInformation;
|
||||
import org.apache.kafka.common.utils.AppInfoParser;
|
||||
|
@ -226,6 +236,8 @@ import org.slf4j.Logger;
|
|||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.InvalidKeyException;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -4071,6 +4083,182 @@ public class KafkaAdminClient extends AdminClient {
|
|||
return new AlterClientQuotasResult(Collections.unmodifiableMap(futures));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
|
||||
final KafkaFutureImpl<DescribeUserScramCredentialsResponseData> dataFuture = new KafkaFutureImpl<>();
|
||||
final long now = time.milliseconds();
|
||||
Call call = new Call("describeUserScramCredentials", calcDeadlineMs(now, options.timeoutMs()),
|
||||
new LeastLoadedNodeProvider()) {
|
||||
@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;
|
||||
DescribeUserScramCredentialsResponseData data = response.data();
|
||||
short messageLevelErrorCode = data.errorCode();
|
||||
if (messageLevelErrorCode != Errors.NONE.code()) {
|
||||
dataFuture.completeExceptionally(Errors.forCode(messageLevelErrorCode).exception(data.errorMessage()));
|
||||
} else {
|
||||
dataFuture.complete(data);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void handleFailure(Throwable throwable) {
|
||||
dataFuture.completeExceptionally(throwable);
|
||||
}
|
||||
};
|
||||
runnable.call(call, now);
|
||||
return new DescribeUserScramCredentialsResult(dataFuture);
|
||||
}
|
||||
|
||||
@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.user(), new KafkaFutureImpl<>());
|
||||
}
|
||||
final Map<String, Exception> userIllegalAlterationExceptions = new HashMap<>();
|
||||
// We need to keep track of users with deletions of an unknown SCRAM mechanism
|
||||
final String usernameMustNotBeEmptyMsg = "Username must not be empty";
|
||||
String passwordMustNotBeEmptyMsg = "Password must not be empty";
|
||||
final String unknownScramMechanismMsg = "Unknown SCRAM mechanism";
|
||||
alterations.stream().filter(a -> a instanceof UserScramCredentialDeletion).forEach(alteration -> {
|
||||
final String user = alteration.user();
|
||||
if (user == null || user.isEmpty()) {
|
||||
userIllegalAlterationExceptions.put(alteration.user(), new UnacceptableCredentialException(usernameMustNotBeEmptyMsg));
|
||||
} else {
|
||||
UserScramCredentialDeletion deletion = (UserScramCredentialDeletion) alteration;
|
||||
ScramMechanism mechanism = deletion.mechanism();
|
||||
if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
|
||||
userIllegalAlterationExceptions.put(user, new UnsupportedSaslMechanismException(unknownScramMechanismMsg));
|
||||
}
|
||||
}
|
||||
});
|
||||
// Creating an upsertion may throw InvalidKeyException or NoSuchAlgorithmException,
|
||||
// so keep track of which users are affected by such a failure so we can fail all their alterations later
|
||||
final Map<String, Map<ScramMechanism, AlterUserScramCredentialsRequestData.ScramCredentialUpsertion>> userInsertions = new HashMap<>();
|
||||
alterations.stream().filter(a -> a instanceof UserScramCredentialUpsertion)
|
||||
.filter(alteration -> !userIllegalAlterationExceptions.containsKey(alteration.user()))
|
||||
.forEach(alteration -> {
|
||||
final String user = alteration.user();
|
||||
if (user == null || user.isEmpty()) {
|
||||
userIllegalAlterationExceptions.put(alteration.user(), new UnacceptableCredentialException(usernameMustNotBeEmptyMsg));
|
||||
} else {
|
||||
UserScramCredentialUpsertion upsertion = (UserScramCredentialUpsertion) alteration;
|
||||
try {
|
||||
byte[] password = upsertion.password();
|
||||
if (password == null || password.length == 0) {
|
||||
userIllegalAlterationExceptions.put(user, new UnacceptableCredentialException(passwordMustNotBeEmptyMsg));
|
||||
} else {
|
||||
ScramMechanism mechanism = upsertion.credentialInfo().mechanism();
|
||||
if (mechanism == null || mechanism == ScramMechanism.UNKNOWN) {
|
||||
userIllegalAlterationExceptions.put(user, new UnsupportedSaslMechanismException(unknownScramMechanismMsg));
|
||||
} else {
|
||||
userInsertions.putIfAbsent(user, new HashMap<>());
|
||||
userInsertions.get(user).put(mechanism, getScramCredentialUpsertion(upsertion));
|
||||
}
|
||||
}
|
||||
} catch (NoSuchAlgorithmException e) {
|
||||
// we might overwrite an exception from a previous alteration, 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, new UnsupportedSaslMechanismException(unknownScramMechanismMsg));
|
||||
} catch (InvalidKeyException e) {
|
||||
// generally shouldn't happen since we deal with the empty password case above,
|
||||
// but we still need to catch/handle it
|
||||
userIllegalAlterationExceptions.put(user, new UnacceptableCredentialException(e.getMessage(), e));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
// submit alterations only for users that do not have an illegal alteration 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.user()))
|
||||
.map(a -> userInsertions.get(a.user()).get(((UserScramCredentialUpsertion) a).credentialInfo().mechanism()))
|
||||
.collect(Collectors.toList()))
|
||||
.setDeletions(alterations.stream()
|
||||
.filter(a -> a instanceof UserScramCredentialDeletion)
|
||||
.filter(a -> !userIllegalAlterationExceptions.containsKey(a.user()))
|
||||
.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);
|
||||
}
|
||||
}
|
||||
/* Now that we have the results for the ones we sent,
|
||||
* fail any users that have an illegal alteration as identified above.
|
||||
* Be sure to do this after the NOT_CONTROLLER error check above
|
||||
* so that all errors are consistent in that case.
|
||||
*/
|
||||
userIllegalAlterationExceptions.entrySet().stream().forEach(entry -> {
|
||||
futures.get(entry.getKey()).completeExceptionally(entry.getValue());
|
||||
});
|
||||
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.user())
|
||||
.setMechanism(u.credentialInfo().mechanism().type())
|
||||
.setIterations(u.credentialInfo().iterations())
|
||||
.setSalt(u.salt())
|
||||
.setSaltedPassword(getSaltedPasword(u.credentialInfo().mechanism(), u.password(), u.salt(), u.credentialInfo().iterations()));
|
||||
}
|
||||
|
||||
private static AlterUserScramCredentialsRequestData.ScramCredentialDeletion getScramCredentialDeletion(UserScramCredentialDeletion d) {
|
||||
return new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(d.user()).setMechanism(d.mechanism().type());
|
||||
}
|
||||
|
||||
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.mechanismName()))
|
||||
.hi(password, salt, iterations);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a sub level error when the request is in batch. If given key was not found,
|
||||
* return an {@link IllegalArgumentException}.
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* 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 mechanism() {
|
||||
return mechanism;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the number of iterations used when creating the credential
|
||||
*/
|
||||
public int iterations() {
|
||||
return iterations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ScramCredentialInfo{" +
|
||||
"mechanism=" + mechanism +
|
||||
", iterations=" + iterations +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
ScramCredentialInfo that = (ScramCredentialInfo) o;
|
||||
return iterations == that.iterations &&
|
||||
mechanism == that.mechanism;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(mechanism, iterations);
|
||||
}
|
||||
}
|
|
@ -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 mechanismName() {
|
||||
return this.mechanismName;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the type indicator for this SASL SCRAM mechanism
|
||||
*/
|
||||
public byte type() {
|
||||
return this.type;
|
||||
}
|
||||
|
||||
private final byte type;
|
||||
private final String mechanismName;
|
||||
|
||||
private ScramMechanism(byte type) {
|
||||
this.type = type;
|
||||
this.mechanismName = toString().replace('_', '-');
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* A request to alter a user's SASL/SCRAM credentials.
|
||||
*
|
||||
* @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 abstract class UserScramCredentialAlteration {
|
||||
protected final String user;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param user the mandatory user
|
||||
*/
|
||||
protected UserScramCredentialAlteration(String user) {
|
||||
this.user = Objects.requireNonNull(user);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the always non-null user
|
||||
*/
|
||||
public String user() {
|
||||
return this.user;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* A request to delete 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 UserScramCredentialDeletion extends UserScramCredentialAlteration {
|
||||
private final ScramMechanism mechanism;
|
||||
|
||||
/**
|
||||
* @param user the mandatory user
|
||||
* @param mechanism the mandatory mechanism
|
||||
*/
|
||||
public UserScramCredentialDeletion(String user, ScramMechanism mechanism) {
|
||||
super(user);
|
||||
this.mechanism = Objects.requireNonNull(mechanism);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the always non-null mechanism
|
||||
*/
|
||||
public ScramMechanism mechanism() {
|
||||
return mechanism;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
* 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.security.scram.internals.ScramFormatter;
|
||||
|
||||
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 ScramFormatter.secureRandomBytes(new SecureRandom());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,82 @@
|
|||
/*
|
||||
* 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, 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 List<ScramCredentialInfo> credentialInfos;
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
UserScramCredentialsDescription that = (UserScramCredentialsDescription) o;
|
||||
return name.equals(that.name) &&
|
||||
credentialInfos.equals(that.credentialInfos);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(name, credentialInfos);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "UserScramCredentialsDescription{" +
|
||||
"name='" + name + '\'' +
|
||||
", credentialInfos=" + credentialInfos +
|
||||
'}';
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @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 = Objects.requireNonNull(name);
|
||||
this.credentialInfos = Collections.unmodifiableList(new ArrayList<>(credentialInfos));
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the user name
|
||||
*/
|
||||
public String name() {
|
||||
return name;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the always non-null/unmodifiable list of SASL/SCRAM credential representations for the user
|
||||
*/
|
||||
public List<ScramCredentialInfo> credentialInfos() {
|
||||
return credentialInfos;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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 that illegally refers to the same resource twice
|
||||
* (for example, trying to both create and delete the same SCRAM credential for a particular user in a single request).
|
||||
*/
|
||||
public class DuplicateResourceException extends ApiException {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final String resource;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param message the exception's message
|
||||
*/
|
||||
public DuplicateResourceException(String message) {
|
||||
this(null, message);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param message the exception's message
|
||||
* @param cause the exception's cause
|
||||
*/
|
||||
public DuplicateResourceException(String message, Throwable cause) {
|
||||
this(null, message, cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param resource the (potentially null) resource that was referred to twice
|
||||
* @param message the exception's message
|
||||
*/
|
||||
public DuplicateResourceException(String resource, String message) {
|
||||
super(message);
|
||||
this.resource = resource;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param resource the (potentially null) resource that was referred to twice
|
||||
* @param message the exception's message
|
||||
* @param cause the exception's cause
|
||||
*/
|
||||
public DuplicateResourceException(String resource, String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
this.resource = resource;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the (potentially null) resource that was referred to twice
|
||||
*/
|
||||
public String resource() {
|
||||
return this.resource;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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 static final long serialVersionUID = 1L;
|
||||
|
||||
private final String resource;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param message the exception's message
|
||||
*/
|
||||
public ResourceNotFoundException(String message) {
|
||||
this(null, message);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param message the exception's message
|
||||
* @param cause the exception's cause
|
||||
*/
|
||||
public ResourceNotFoundException(String message, Throwable cause) {
|
||||
this(null, message, cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param resource the (potentially null) resource that was not found
|
||||
* @param message the exception's message
|
||||
*/
|
||||
public ResourceNotFoundException(String resource, String message) {
|
||||
super(message);
|
||||
this.resource = resource;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param resource the (potentially null) resource that was not found
|
||||
* @param message the exception's message
|
||||
* @param cause the exception's cause
|
||||
*/
|
||||
public ResourceNotFoundException(String resource, String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
this.resource = resource;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the (potentially null) resource that was not found
|
||||
*/
|
||||
public String resource() {
|
||||
return this.resource;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* 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 when attempting to define a credential that does not meet the criteria for acceptability
|
||||
* (for example, attempting to create a SCRAM credential with an empty username or password or too few/many iterations).
|
||||
*/
|
||||
public class UnacceptableCredentialException extends ApiException {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
*
|
||||
* @param message the exception's message
|
||||
*/
|
||||
public UnacceptableCredentialException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param message the exception's message
|
||||
* @param cause the exception's cause
|
||||
*/
|
||||
public UnacceptableCredentialException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
}
|
|
@ -22,6 +22,8 @@ import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData;
|
|||
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData;
|
||||
import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData;
|
||||
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData;
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.message.ApiMessageType;
|
||||
import org.apache.kafka.common.message.AddOffsetsToTxnRequestData;
|
||||
import org.apache.kafka.common.message.AddOffsetsToTxnResponseData;
|
||||
|
@ -61,6 +63,8 @@ import org.apache.kafka.common.message.DescribeGroupsRequestData;
|
|||
import org.apache.kafka.common.message.DescribeGroupsResponseData;
|
||||
import org.apache.kafka.common.message.DescribeLogDirsRequestData;
|
||||
import org.apache.kafka.common.message.DescribeLogDirsResponseData;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.message.ElectLeadersRequestData;
|
||||
import org.apache.kafka.common.message.ElectLeadersResponseData;
|
||||
import org.apache.kafka.common.message.EndTxnRequestData;
|
||||
|
@ -217,7 +221,11 @@ public enum ApiKeys {
|
|||
DESCRIBE_CLIENT_QUOTAS(48, "DescribeClientQuotas", DescribeClientQuotasRequestData.SCHEMAS,
|
||||
DescribeClientQuotasResponseData.SCHEMAS),
|
||||
ALTER_CLIENT_QUOTAS(49, "AlterClientQuotas", AlterClientQuotasRequestData.SCHEMAS,
|
||||
AlterClientQuotasResponseData.SCHEMAS);
|
||||
AlterClientQuotasResponseData.SCHEMAS),
|
||||
DESCRIBE_USER_SCRAM_CREDENTIALS(50, "DescribeUserScramCredentials", DescribeUserScramCredentialsRequestData.SCHEMAS,
|
||||
DescribeUserScramCredentialsResponseData.SCHEMAS),
|
||||
ALTER_USER_SCRAM_CREDENTIALS(51, "AlterUserScramCredentials", AlterUserScramCredentialsRequestData.SCHEMAS,
|
||||
AlterUserScramCredentialsResponseData.SCHEMAS);
|
||||
|
||||
private static final ApiKeys[] ID_TO_TYPE;
|
||||
private static final int MIN_API_KEY = 0;
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.kafka.common.errors.ApiException;
|
|||
import org.apache.kafka.common.errors.BrokerNotAvailableException;
|
||||
import org.apache.kafka.common.errors.ClusterAuthorizationException;
|
||||
import org.apache.kafka.common.errors.ConcurrentTransactionsException;
|
||||
import org.apache.kafka.common.errors.DuplicateResourceException;
|
||||
import org.apache.kafka.common.errors.GroupSubscribedToTopicException;
|
||||
import org.apache.kafka.common.errors.ControllerMovedException;
|
||||
import org.apache.kafka.common.errors.CoordinatorLoadInProgressException;
|
||||
|
@ -79,7 +80,9 @@ import org.apache.kafka.common.errors.OffsetNotAvailableException;
|
|||
import org.apache.kafka.common.errors.OffsetOutOfRangeException;
|
||||
import org.apache.kafka.common.errors.OperationNotAttemptedException;
|
||||
import org.apache.kafka.common.errors.OutOfOrderSequenceException;
|
||||
import org.apache.kafka.common.errors.ResourceNotFoundException;
|
||||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
|
||||
import org.apache.kafka.common.errors.UnacceptableCredentialException;
|
||||
import org.apache.kafka.common.errors.UnstableOffsetCommitException;
|
||||
import org.apache.kafka.common.errors.PolicyViolationException;
|
||||
import org.apache.kafka.common.errors.PreferredLeaderNotAvailableException;
|
||||
|
@ -325,7 +328,10 @@ public enum Errors {
|
|||
UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new),
|
||||
THROTTLING_QUOTA_EXCEEDED(89, "The throttling quota has been exceeded.", ThrottlingQuotaExceededException::new),
|
||||
PRODUCER_FENCED(90, "There is a newer producer with the same transactionalId " +
|
||||
"which fences the current one.", ProducerFencedException::new);
|
||||
"which fences the current one.", ProducerFencedException::new),
|
||||
RESOURCE_NOT_FOUND(91, "A request illegally referred to a resource that does not exist.", ResourceNotFoundException::new),
|
||||
DUPLICATE_RESOURCE(92, "A request illegally referred to the same resource twice.", DuplicateResourceException::new),
|
||||
UNACCEPTABLE_CREDENTIAL(93, "Requested credential would not meet criteria for acceptability.", UnacceptableCredentialException::new);
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(Errors.class);
|
||||
|
||||
|
|
|
@ -244,6 +244,10 @@ public abstract class AbstractRequest implements AbstractRequestResponse {
|
|||
return new DescribeClientQuotasRequest(struct, apiVersion);
|
||||
case ALTER_CLIENT_QUOTAS:
|
||||
return new AlterClientQuotasRequest(struct, apiVersion);
|
||||
case DESCRIBE_USER_SCRAM_CREDENTIALS:
|
||||
return new DescribeUserScramCredentialsRequest(struct, apiVersion);
|
||||
case ALTER_USER_SCRAM_CREDENTIALS:
|
||||
return new AlterUserScramCredentialsRequest(struct, apiVersion);
|
||||
default:
|
||||
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " +
|
||||
"code should be updated to do so.", apiKey));
|
||||
|
|
|
@ -187,6 +187,10 @@ public abstract class AbstractResponse implements AbstractRequestResponse {
|
|||
return new DescribeClientQuotasResponse(struct, version);
|
||||
case ALTER_CLIENT_QUOTAS:
|
||||
return new AlterClientQuotasResponse(struct, version);
|
||||
case DESCRIBE_USER_SCRAM_CREDENTIALS:
|
||||
return new DescribeUserScramCredentialsResponse(struct, version);
|
||||
case ALTER_USER_SCRAM_CREDENTIALS:
|
||||
return new AlterUserScramCredentialsResponse(struct, version);
|
||||
default:
|
||||
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " +
|
||||
"code should be updated to do so.", apiKey));
|
||||
|
|
|
@ -0,0 +1,102 @@
|
|||
/*
|
||||
* 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.requests;
|
||||
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.types.Struct;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class AlterUserScramCredentialsRequest extends AbstractRequest {
|
||||
|
||||
public static class Builder extends AbstractRequest.Builder<AlterUserScramCredentialsRequest> {
|
||||
private final AlterUserScramCredentialsRequestData data;
|
||||
|
||||
public Builder(AlterUserScramCredentialsRequestData data) {
|
||||
super(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AlterUserScramCredentialsRequest build(short version) {
|
||||
return new AlterUserScramCredentialsRequest(data, version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return data.toString();
|
||||
}
|
||||
}
|
||||
|
||||
private AlterUserScramCredentialsRequestData data;
|
||||
private final short version;
|
||||
|
||||
private AlterUserScramCredentialsRequest(AlterUserScramCredentialsRequestData data, short version) {
|
||||
super(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS, version);
|
||||
this.data = data;
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
AlterUserScramCredentialsRequest(Struct struct, short version) {
|
||||
super(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS, version);
|
||||
this.data = new AlterUserScramCredentialsRequestData(struct, version);
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
public static AlterUserScramCredentialsRequest parse(ByteBuffer buffer, short version) {
|
||||
return new AlterUserScramCredentialsRequest(
|
||||
ApiKeys.ALTER_USER_SCRAM_CREDENTIALS.parseRequest(version, buffer), version
|
||||
);
|
||||
}
|
||||
|
||||
public AlterUserScramCredentialsRequestData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
/**
|
||||
* Visible for testing.
|
||||
*/
|
||||
@Override
|
||||
public Struct toStruct() {
|
||||
return data.toStruct(version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
|
||||
ApiError apiError = ApiError.fromThrowable(e);
|
||||
short errorCode = apiError.error().code();
|
||||
String errorMessage = apiError.message();
|
||||
Set<String> users = Stream.concat(
|
||||
this.data.deletions().stream().map(deletion -> deletion.name()),
|
||||
this.data.upsertions().stream().map(upsertion -> upsertion.name()))
|
||||
.collect(Collectors.toSet());
|
||||
List<AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult> results =
|
||||
users.stream().sorted().map(user ->
|
||||
new AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(errorCode)
|
||||
.setErrorMessage(errorMessage))
|
||||
.collect(Collectors.toList());
|
||||
return new AlterUserScramCredentialsResponse(new AlterUserScramCredentialsResponseData().setResults(results));
|
||||
}
|
||||
}
|
|
@ -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.common.requests;
|
||||
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.protocol.types.Struct;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
|
||||
public class AlterUserScramCredentialsResponse extends AbstractResponse {
|
||||
|
||||
private final AlterUserScramCredentialsResponseData data;
|
||||
|
||||
public AlterUserScramCredentialsResponse(Struct struct) {
|
||||
this(struct, ApiKeys.ALTER_USER_SCRAM_CREDENTIALS.latestVersion());
|
||||
}
|
||||
|
||||
public AlterUserScramCredentialsResponse(AlterUserScramCredentialsResponseData responseData) {
|
||||
this.data = responseData;
|
||||
}
|
||||
|
||||
AlterUserScramCredentialsResponse(Struct struct, short version) {
|
||||
this.data = new AlterUserScramCredentialsResponseData(struct, version);
|
||||
}
|
||||
|
||||
public static AlterUserScramCredentialsResponse parse(ByteBuffer buffer, short version) {
|
||||
return new AlterUserScramCredentialsResponse(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS.responseSchema(version).read(buffer), version);
|
||||
}
|
||||
|
||||
public AlterUserScramCredentialsResponseData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldClientThrottle(short version) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int throttleTimeMs() {
|
||||
return data.throttleTimeMs();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode())));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Struct toStruct(short version) {
|
||||
return data.toStruct(version);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* 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.requests;
|
||||
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.types.Struct;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class DescribeUserScramCredentialsRequest extends AbstractRequest {
|
||||
|
||||
public static class Builder extends AbstractRequest.Builder<DescribeUserScramCredentialsRequest> {
|
||||
private final DescribeUserScramCredentialsRequestData data;
|
||||
|
||||
public Builder(DescribeUserScramCredentialsRequestData data) {
|
||||
super(ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS);
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DescribeUserScramCredentialsRequest build(short version) {
|
||||
return new DescribeUserScramCredentialsRequest(data, version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return data.toString();
|
||||
}
|
||||
}
|
||||
|
||||
private DescribeUserScramCredentialsRequestData data;
|
||||
private final short version;
|
||||
|
||||
private DescribeUserScramCredentialsRequest(DescribeUserScramCredentialsRequestData data, short version) {
|
||||
super(ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS, version);
|
||||
this.data = data;
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
DescribeUserScramCredentialsRequest(Struct struct, short version) {
|
||||
super(ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS, version);
|
||||
this.data = new DescribeUserScramCredentialsRequestData(struct, version);
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
public static DescribeUserScramCredentialsRequest parse(ByteBuffer buffer, short version) {
|
||||
return new DescribeUserScramCredentialsRequest(
|
||||
ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS.parseRequest(version, buffer), version
|
||||
);
|
||||
}
|
||||
|
||||
public DescribeUserScramCredentialsRequestData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
/**
|
||||
* Visible for testing.
|
||||
*/
|
||||
@Override
|
||||
public Struct toStruct() {
|
||||
return data.toStruct(version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
|
||||
ApiError apiError = ApiError.fromThrowable(e);
|
||||
return new DescribeUserScramCredentialsResponse(new DescribeUserScramCredentialsResponseData()
|
||||
.setErrorCode(apiError.error().code())
|
||||
.setErrorMessage(apiError.message()));
|
||||
}
|
||||
}
|
|
@ -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.common.requests;
|
||||
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.protocol.types.Struct;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
|
||||
public class DescribeUserScramCredentialsResponse extends AbstractResponse {
|
||||
|
||||
private final DescribeUserScramCredentialsResponseData data;
|
||||
|
||||
public DescribeUserScramCredentialsResponse(Struct struct) {
|
||||
this(struct, ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS.latestVersion());
|
||||
}
|
||||
|
||||
public DescribeUserScramCredentialsResponse(DescribeUserScramCredentialsResponseData responseData) {
|
||||
this.data = responseData;
|
||||
}
|
||||
|
||||
DescribeUserScramCredentialsResponse(Struct struct, short version) {
|
||||
this.data = new DescribeUserScramCredentialsResponseData(struct, version);
|
||||
}
|
||||
|
||||
public static DescribeUserScramCredentialsResponse parse(ByteBuffer buffer, short version) {
|
||||
return new DescribeUserScramCredentialsResponse(ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS.responseSchema(version).read(buffer), version);
|
||||
}
|
||||
|
||||
public DescribeUserScramCredentialsResponseData data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldClientThrottle(short version) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int throttleTimeMs() {
|
||||
return data.throttleTimeMs();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode())));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Struct toStruct(short version) {
|
||||
return data.toStruct(version);
|
||||
}
|
||||
}
|
|
@ -63,7 +63,7 @@ public class ScramFormatter {
|
|||
return messageDigest.digest(str);
|
||||
}
|
||||
|
||||
public byte[] xor(byte[] first, byte[] second) {
|
||||
public static byte[] xor(byte[] first, byte[] second) {
|
||||
if (first.length != second.length)
|
||||
throw new IllegalArgumentException("Argument arrays must be of the same length");
|
||||
byte[] result = new byte[first.length];
|
||||
|
@ -86,7 +86,7 @@ public class ScramFormatter {
|
|||
return result;
|
||||
}
|
||||
|
||||
public byte[] normalize(String str) {
|
||||
public static byte[] normalize(String str) {
|
||||
return toBytes(str);
|
||||
}
|
||||
|
||||
|
@ -102,12 +102,12 @@ public class ScramFormatter {
|
|||
return hash(clientKey);
|
||||
}
|
||||
|
||||
public String saslName(String username) {
|
||||
public static String saslName(String username) {
|
||||
String replace1 = EQUAL.matcher(username).replaceAll(Matcher.quoteReplacement("=3D"));
|
||||
return COMMA.matcher(replace1).replaceAll(Matcher.quoteReplacement("=2C"));
|
||||
}
|
||||
|
||||
public String username(String saslName) {
|
||||
public static String username(String saslName) {
|
||||
String username = EQUAL_TWO_C.matcher(saslName).replaceAll(Matcher.quoteReplacement(","));
|
||||
if (EQUAL_THREE_D.matcher(username).replaceAll(Matcher.quoteReplacement("")).indexOf('=') >= 0) {
|
||||
throw new IllegalArgumentException("Invalid username: " + saslName);
|
||||
|
@ -115,7 +115,7 @@ public class ScramFormatter {
|
|||
return EQUAL_THREE_D.matcher(username).replaceAll(Matcher.quoteReplacement("="));
|
||||
}
|
||||
|
||||
public String authMessage(String clientFirstMessageBare, String serverFirstMessage, String clientFinalMessageWithoutProof) {
|
||||
public static String authMessage(String clientFirstMessageBare, String serverFirstMessage, String clientFinalMessageWithoutProof) {
|
||||
return clientFirstMessageBare + "," + serverFirstMessage + "," + clientFinalMessageWithoutProof;
|
||||
}
|
||||
|
||||
|
@ -151,14 +151,22 @@ public class ScramFormatter {
|
|||
}
|
||||
|
||||
public String secureRandomString() {
|
||||
return secureRandomString(random);
|
||||
}
|
||||
|
||||
public static String secureRandomString(SecureRandom random) {
|
||||
return new BigInteger(130, random).toString(Character.MAX_RADIX);
|
||||
}
|
||||
|
||||
public byte[] secureRandomBytes() {
|
||||
return toBytes(secureRandomString());
|
||||
return secureRandomBytes(random);
|
||||
}
|
||||
|
||||
public byte[] toBytes(String str) {
|
||||
public static byte[] secureRandomBytes(SecureRandom random) {
|
||||
return toBytes(secureRandomString(random));
|
||||
}
|
||||
|
||||
public static byte[] toBytes(String str) {
|
||||
return str.getBytes(StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
|
@ -166,6 +174,14 @@ public class ScramFormatter {
|
|||
try {
|
||||
byte[] salt = secureRandomBytes();
|
||||
byte[] saltedPassword = saltedPassword(password, salt, iterations);
|
||||
return generateCredential(salt, saltedPassword, iterations);
|
||||
} catch (InvalidKeyException e) {
|
||||
throw new KafkaException("Could not create credential", e);
|
||||
}
|
||||
}
|
||||
|
||||
public ScramCredential generateCredential(byte[] salt, byte[] saltedPassword, int iterations) {
|
||||
try {
|
||||
byte[] clientKey = clientKey(saltedPassword);
|
||||
byte[] storedKey = storedKey(clientKey);
|
||||
byte[] serverKey = serverKey(saltedPassword);
|
||||
|
|
|
@ -112,7 +112,7 @@ public class ScramSaslClient implements SaslClient {
|
|||
}
|
||||
|
||||
String username = nameCallback.getName();
|
||||
String saslName = formatter.saslName(username);
|
||||
String saslName = ScramFormatter.saslName(username);
|
||||
Map<String, String> extensions = extensionsCallback.extensions();
|
||||
this.clientFirstMessage = new ScramMessages.ClientFirstMessage(saslName, clientNonce, extensions);
|
||||
setState(State.RECEIVE_SERVER_FIRST_MESSAGE);
|
||||
|
@ -188,7 +188,7 @@ public class ScramSaslClient implements SaslClient {
|
|||
|
||||
private ClientFinalMessage handleServerFirstMessage(char[] password) throws SaslException {
|
||||
try {
|
||||
byte[] passwordBytes = formatter.normalize(new String(password));
|
||||
byte[] passwordBytes = ScramFormatter.normalize(new String(password));
|
||||
this.saltedPassword = formatter.hi(passwordBytes, serverFirstMessage.salt(), serverFirstMessage.iterations());
|
||||
|
||||
ClientFinalMessage clientFinalMessage = new ClientFinalMessage("n,,".getBytes(StandardCharsets.UTF_8), serverFirstMessage.nonce());
|
||||
|
|
|
@ -107,7 +107,7 @@ public class ScramSaslServer implements SaslServer {
|
|||
String serverNonce = formatter.secureRandomString();
|
||||
try {
|
||||
String saslName = clientFirstMessage.saslName();
|
||||
this.username = formatter.username(saslName);
|
||||
this.username = ScramFormatter.username(saslName);
|
||||
NameCallback nameCallback = new NameCallback("username", username);
|
||||
ScramCredentialCallback credentialCallback;
|
||||
if (scramExtensions.tokenAuthenticated()) {
|
||||
|
|
|
@ -0,0 +1,44 @@
|
|||
// 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.
|
||||
|
||||
{
|
||||
"apiKey": 51,
|
||||
"type": "request",
|
||||
"name": "AlterUserScramCredentialsRequest",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "Deletions", "type": "[]ScramCredentialDeletion", "versions": "0+",
|
||||
"about": "The SCRAM credentials to remove.", "fields": [
|
||||
{ "name": "Name", "type": "string", "versions": "0+",
|
||||
"about": "The user name." },
|
||||
{ "name": "Mechanism", "type": "int8", "versions": "0+",
|
||||
"about": "The SCRAM mechanism." }
|
||||
]},
|
||||
{ "name": "Upsertions", "type": "[]ScramCredentialUpsertion", "versions": "0+",
|
||||
"about": "The SCRAM credentials to update/insert.", "fields": [
|
||||
{ "name": "Name", "type": "string", "versions": "0+",
|
||||
"about": "The user name." },
|
||||
{ "name": "Mechanism", "type": "int8", "versions": "0+",
|
||||
"about": "The SCRAM mechanism." },
|
||||
{ "name": "Iterations", "type": "int32", "versions": "0+",
|
||||
"about": "The number of iterations." },
|
||||
{ "name": "Salt", "type": "bytes", "versions": "0+",
|
||||
"about": "A random salt generated by the client." },
|
||||
{ "name": "SaltedPassword", "type": "bytes", "versions": "0+",
|
||||
"about": "The salted password." }
|
||||
]}
|
||||
]
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
// 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.
|
||||
|
||||
{
|
||||
"apiKey": 51,
|
||||
"type": "response",
|
||||
"name": "AlterUserScramCredentialsResponse",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
"about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
|
||||
{ "name": "Results", "type": "[]AlterUserScramCredentialsResult", "versions": "0+",
|
||||
"about": "The results for deletions and alterations, one per affected user.", "fields": [
|
||||
{ "name": "User", "type": "string", "versions": "0+",
|
||||
"about": "The user name." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The error code." },
|
||||
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
|
||||
"about": "The error message, if any." }
|
||||
]}
|
||||
]
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
// 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.
|
||||
|
||||
{
|
||||
"apiKey": 50,
|
||||
"type": "request",
|
||||
"name": "DescribeUserScramCredentialsRequest",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "Users", "type": "[]UserName", "versions": "0+", "nullableVersions": "0+",
|
||||
"about": "The users to describe, or null/empty to describe all users.", "fields": [
|
||||
{ "name": "Name", "type": "string", "versions": "0+",
|
||||
"about": "The user name." }
|
||||
]}
|
||||
]
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
// 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.
|
||||
|
||||
{
|
||||
"apiKey": 50,
|
||||
"type": "response",
|
||||
"name": "DescribeUserScramCredentialsResponse",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
"about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The message-level error code, 0 except for user authorization or infrastructure issues." },
|
||||
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
|
||||
"about": "The message-level error message, if any." },
|
||||
{ "name": "Results", "type": "[]DescribeUserScramCredentialsResult", "versions": "0+",
|
||||
"about": "The results for descriptions, one per user.", "fields": [
|
||||
{ "name": "User", "type": "string", "versions": "0+",
|
||||
"about": "The user name." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The user-level error code." },
|
||||
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
|
||||
"about": "The user-level error message, if any." },
|
||||
{ "name": "CredentialInfos", "type": "[]CredentialInfo", "versions": "0+",
|
||||
"about": "The mechanism and related information associated with the user's SCRAM credentials.", "fields": [
|
||||
{ "name": "Mechanism", "type": "int8", "versions": "0+",
|
||||
"about": "The SCRAM mechanism." },
|
||||
{ "name": "Iterations", "type": "int32", "versions": "0+",
|
||||
"about": "The number of iterations used in the SCRAM credential." }]}
|
||||
]}
|
||||
]
|
||||
}
|
|
@ -0,0 +1,118 @@
|
|||
/*
|
||||
* 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.internals.KafkaFutureImpl;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class DescribeUserScramCredentialsResultTest {
|
||||
@Test
|
||||
public void testTopLevelError() {
|
||||
KafkaFutureImpl<DescribeUserScramCredentialsResponseData> dataFuture = new KafkaFutureImpl<>();
|
||||
dataFuture.completeExceptionally(new RuntimeException());
|
||||
DescribeUserScramCredentialsResult results = new DescribeUserScramCredentialsResult(dataFuture);
|
||||
try {
|
||||
results.all().get();
|
||||
fail("expected all() to fail when there is a top-level error");
|
||||
} catch (Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
try {
|
||||
results.users().get();
|
||||
fail("expected users() to fail when there is a top-level error");
|
||||
} catch (Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
try {
|
||||
results.description("whatever").get();
|
||||
fail("expected description() to fail when there is a top-level error");
|
||||
} catch (Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUserLevelErrors() throws Exception {
|
||||
String goodUser = "goodUser";
|
||||
String unknownUser = "unknownUser";
|
||||
String failedUser = "failedUser";
|
||||
KafkaFutureImpl<DescribeUserScramCredentialsResponseData> dataFuture = new KafkaFutureImpl<>();
|
||||
ScramMechanism scramSha256 = ScramMechanism.SCRAM_SHA_256;
|
||||
int iterations = 4096;
|
||||
dataFuture.complete(new DescribeUserScramCredentialsResponseData().setErrorCode(Errors.NONE.code()).setResults(Arrays.asList(
|
||||
new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult().setUser(goodUser).setCredentialInfos(
|
||||
Arrays.asList(new DescribeUserScramCredentialsResponseData.CredentialInfo().setMechanism(scramSha256.type()).setIterations(iterations))),
|
||||
new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult().setUser(unknownUser).setErrorCode(Errors.RESOURCE_NOT_FOUND.code()),
|
||||
new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult().setUser(failedUser).setErrorCode(Errors.DUPLICATE_RESOURCE.code()))));
|
||||
DescribeUserScramCredentialsResult results = new DescribeUserScramCredentialsResult(dataFuture);
|
||||
try {
|
||||
results.all().get();
|
||||
fail("expected all() to fail when there is a user-level error");
|
||||
} catch (Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
assertEquals("Expected 2 users with credentials", Arrays.asList(goodUser, failedUser), results.users().get());
|
||||
UserScramCredentialsDescription goodUserDescription = results.description(goodUser).get();
|
||||
assertEquals(new UserScramCredentialsDescription(goodUser, Arrays.asList(new ScramCredentialInfo(scramSha256, iterations))), goodUserDescription);
|
||||
try {
|
||||
results.description(failedUser).get();
|
||||
fail("expected description(failedUser) to fail when there is a user-level error");
|
||||
} catch (Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
try {
|
||||
results.description(unknownUser).get();
|
||||
fail("expected description(unknownUser) to fail when there is no such user");
|
||||
} catch (Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessfulDescription() throws Exception {
|
||||
String goodUser = "goodUser";
|
||||
String unknownUser = "unknownUser";
|
||||
KafkaFutureImpl<DescribeUserScramCredentialsResponseData> dataFuture = new KafkaFutureImpl<>();
|
||||
ScramMechanism scramSha256 = ScramMechanism.SCRAM_SHA_256;
|
||||
int iterations = 4096;
|
||||
dataFuture.complete(new DescribeUserScramCredentialsResponseData().setErrorCode(Errors.NONE.code()).setResults(Arrays.asList(
|
||||
new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult().setUser(goodUser).setCredentialInfos(
|
||||
Arrays.asList(new DescribeUserScramCredentialsResponseData.CredentialInfo().setMechanism(scramSha256.type()).setIterations(iterations))))));
|
||||
DescribeUserScramCredentialsResult results = new DescribeUserScramCredentialsResult(dataFuture);
|
||||
assertEquals("Expected 1 user with credentials", Arrays.asList(goodUser), results.users().get());
|
||||
Map<String, UserScramCredentialsDescription> allResults = results.all().get();
|
||||
assertEquals(1, allResults.size());
|
||||
UserScramCredentialsDescription goodUserDescriptionViaAll = allResults.get(goodUser);
|
||||
assertEquals(new UserScramCredentialsDescription(goodUser, Arrays.asList(new ScramCredentialInfo(scramSha256, iterations))), goodUserDescriptionViaAll);
|
||||
assertEquals("Expected same thing via all() and description()", goodUserDescriptionViaAll, results.description(goodUser).get());
|
||||
try {
|
||||
results.description(unknownUser).get();
|
||||
fail("expected description(unknownUser) to fail when there is no such user even when all() succeeds");
|
||||
} catch (Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
}
|
||||
}
|
|
@ -70,6 +70,7 @@ import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData;
|
|||
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData;
|
||||
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirPartitionResult;
|
||||
import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult;
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.message.CreatePartitionsResponseData;
|
||||
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult;
|
||||
import org.apache.kafka.common.message.CreateAclsResponseData;
|
||||
|
@ -90,6 +91,8 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData;
|
|||
import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember;
|
||||
import org.apache.kafka.common.message.DescribeLogDirsResponseData;
|
||||
import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo;
|
||||
import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
|
||||
import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
|
||||
import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData;
|
||||
|
@ -115,6 +118,7 @@ import org.apache.kafka.common.quota.ClientQuotaFilterComponent;
|
|||
import org.apache.kafka.common.requests.AlterClientQuotasResponse;
|
||||
import org.apache.kafka.common.requests.AlterPartitionReassignmentsResponse;
|
||||
import org.apache.kafka.common.requests.AlterReplicaLogDirsResponse;
|
||||
import org.apache.kafka.common.requests.AlterUserScramCredentialsResponse;
|
||||
import org.apache.kafka.common.requests.ApiError;
|
||||
import org.apache.kafka.common.requests.CreateAclsResponse;
|
||||
import org.apache.kafka.common.requests.CreatePartitionsRequest;
|
||||
|
@ -131,6 +135,7 @@ import org.apache.kafka.common.requests.DescribeClientQuotasResponse;
|
|||
import org.apache.kafka.common.requests.DescribeConfigsResponse;
|
||||
import org.apache.kafka.common.requests.DescribeGroupsResponse;
|
||||
import org.apache.kafka.common.requests.DescribeLogDirsResponse;
|
||||
import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse;
|
||||
import org.apache.kafka.common.requests.ElectLeadersResponse;
|
||||
import org.apache.kafka.common.requests.FindCoordinatorResponse;
|
||||
import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse;
|
||||
|
@ -4413,6 +4418,155 @@ public class KafkaAdminClientTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDescribeUserScramCredentials() throws Exception {
|
||||
try (AdminClientUnitTestEnv env = mockClientEnv()) {
|
||||
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
||||
|
||||
final String user0Name = "user0";
|
||||
ScramMechanism user0ScramMechanism0 = ScramMechanism.SCRAM_SHA_256;
|
||||
int user0Iterations0 = 4096;
|
||||
ScramMechanism user0ScramMechanism1 = ScramMechanism.SCRAM_SHA_512;
|
||||
int user0Iterations1 = 8192;
|
||||
|
||||
final CredentialInfo user0CredentialInfo0 = new CredentialInfo();
|
||||
user0CredentialInfo0.setMechanism(user0ScramMechanism0.type());
|
||||
user0CredentialInfo0.setIterations(user0Iterations0);
|
||||
final CredentialInfo user0CredentialInfo1 = new CredentialInfo();
|
||||
user0CredentialInfo1.setMechanism(user0ScramMechanism1.type());
|
||||
user0CredentialInfo1.setIterations(user0Iterations1);
|
||||
|
||||
final String user1Name = "user1";
|
||||
ScramMechanism user1ScramMechanism = ScramMechanism.SCRAM_SHA_256;
|
||||
int user1Iterations = 4096;
|
||||
|
||||
final CredentialInfo user1CredentialInfo = new CredentialInfo();
|
||||
user1CredentialInfo.setMechanism(user1ScramMechanism.type());
|
||||
user1CredentialInfo.setIterations(user1Iterations);
|
||||
|
||||
DescribeUserScramCredentialsResponseData responseData = new DescribeUserScramCredentialsResponseData();
|
||||
responseData.setResults(Arrays.asList(
|
||||
new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult()
|
||||
.setUser(user0Name)
|
||||
.setCredentialInfos(Arrays.asList(user0CredentialInfo0, user0CredentialInfo1)),
|
||||
new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult()
|
||||
.setUser(user1Name)
|
||||
.setCredentialInfos(Arrays.asList(user1CredentialInfo))));
|
||||
|
||||
env.kafkaClient().prepareResponse(new DescribeUserScramCredentialsResponse(responseData));
|
||||
|
||||
List<String> usersRequestedList = asList(user0Name, user1Name);
|
||||
Set<String> usersRequestedSet = usersRequestedList.stream().collect(Collectors.toSet());
|
||||
DescribeUserScramCredentialsResult result = env.adminClient().describeUserScramCredentials(usersRequestedList);
|
||||
Map<String, UserScramCredentialsDescription> descriptionResults = result.all().get();
|
||||
KafkaFuture<UserScramCredentialsDescription> user0DescriptionFuture = result.description(user0Name);
|
||||
KafkaFuture<UserScramCredentialsDescription> user1DescriptionFuture = result.description(user1Name);
|
||||
Set<String> usersDescribedFromUsersSet = result.users().get().stream().collect(Collectors.toSet());
|
||||
assertEquals(usersRequestedSet, usersDescribedFromUsersSet);
|
||||
Set<String> usersDescribedFromMapKeySet = descriptionResults.keySet();
|
||||
assertEquals(usersRequestedSet, usersDescribedFromMapKeySet);
|
||||
|
||||
UserScramCredentialsDescription userScramCredentialsDescription0 = descriptionResults.get(user0Name);
|
||||
assertEquals(user0Name, userScramCredentialsDescription0.name());
|
||||
assertEquals(2, userScramCredentialsDescription0.credentialInfos().size());
|
||||
assertEquals(user0ScramMechanism0, userScramCredentialsDescription0.credentialInfos().get(0).mechanism());
|
||||
assertEquals(user0Iterations0, userScramCredentialsDescription0.credentialInfos().get(0).iterations());
|
||||
assertEquals(user0ScramMechanism1, userScramCredentialsDescription0.credentialInfos().get(1).mechanism());
|
||||
assertEquals(user0Iterations1, userScramCredentialsDescription0.credentialInfos().get(1).iterations());
|
||||
assertEquals(userScramCredentialsDescription0, user0DescriptionFuture.get());
|
||||
|
||||
UserScramCredentialsDescription userScramCredentialsDescription1 = descriptionResults.get(user1Name);
|
||||
assertEquals(user1Name, userScramCredentialsDescription1.name());
|
||||
assertEquals(1, userScramCredentialsDescription1.credentialInfos().size());
|
||||
assertEquals(user1ScramMechanism, userScramCredentialsDescription1.credentialInfos().get(0).mechanism());
|
||||
assertEquals(user1Iterations, userScramCredentialsDescription1.credentialInfos().get(0).iterations());
|
||||
assertEquals(userScramCredentialsDescription1, user1DescriptionFuture.get());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlterUserScramCredentialsUnknownMechanism() throws Exception {
|
||||
try (AdminClientUnitTestEnv env = mockClientEnv()) {
|
||||
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
||||
|
||||
final String user0Name = "user0";
|
||||
ScramMechanism user0ScramMechanism0 = ScramMechanism.UNKNOWN;
|
||||
|
||||
final String user1Name = "user1";
|
||||
ScramMechanism user1ScramMechanism0 = ScramMechanism.UNKNOWN;
|
||||
|
||||
final String user2Name = "user2";
|
||||
ScramMechanism user2ScramMechanism0 = ScramMechanism.SCRAM_SHA_256;
|
||||
|
||||
AlterUserScramCredentialsResponseData responseData = new AlterUserScramCredentialsResponseData();
|
||||
responseData.setResults(Arrays.asList(
|
||||
new AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult().setUser(user2Name)));
|
||||
|
||||
env.kafkaClient().prepareResponse(new AlterUserScramCredentialsResponse(responseData));
|
||||
|
||||
AlterUserScramCredentialsResult result = env.adminClient().alterUserScramCredentials(Arrays.asList(
|
||||
new UserScramCredentialDeletion(user0Name, user0ScramMechanism0),
|
||||
new UserScramCredentialUpsertion(user1Name, new ScramCredentialInfo(user1ScramMechanism0, 8192), "password"),
|
||||
new UserScramCredentialUpsertion(user2Name, new ScramCredentialInfo(user2ScramMechanism0, 4096), "password")));
|
||||
Map<String, KafkaFuture<Void>> resultData = result.values();
|
||||
assertEquals(3, resultData.size());
|
||||
Arrays.asList(user0Name, user1Name).stream().forEach(u -> {
|
||||
assertTrue(resultData.containsKey(u));
|
||||
try {
|
||||
resultData.get(u).get();
|
||||
fail("Expected request for user " + u + " to complete exceptionally, but it did not");
|
||||
} catch (Exception expected) {
|
||||
// ignore
|
||||
}
|
||||
});
|
||||
assertTrue(resultData.containsKey(user2Name));
|
||||
try {
|
||||
resultData.get(user2Name).get();
|
||||
} catch (Exception e) {
|
||||
fail("Expected request for user " + user2Name + " to NOT complete excdptionally, but it did");
|
||||
}
|
||||
try {
|
||||
result.all().get();
|
||||
fail("Expected 'result.all().get()' to throw an exception since at least one user failed, but it did not");
|
||||
} catch (final Exception expected) {
|
||||
// ignore, expected
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlterUserScramCredentials() throws Exception {
|
||||
try (AdminClientUnitTestEnv env = mockClientEnv()) {
|
||||
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
||||
|
||||
final String user0Name = "user0";
|
||||
ScramMechanism user0ScramMechanism0 = ScramMechanism.SCRAM_SHA_256;
|
||||
ScramMechanism user0ScramMechanism1 = ScramMechanism.SCRAM_SHA_512;
|
||||
final String user1Name = "user1";
|
||||
ScramMechanism user1ScramMechanism0 = ScramMechanism.SCRAM_SHA_256;
|
||||
final String user2Name = "user2";
|
||||
ScramMechanism user2ScramMechanism0 = ScramMechanism.SCRAM_SHA_512;
|
||||
AlterUserScramCredentialsResponseData responseData = new AlterUserScramCredentialsResponseData();
|
||||
responseData.setResults(Arrays.asList(user0Name, user1Name, user2Name).stream().map(u ->
|
||||
new AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult()
|
||||
.setUser(u).setErrorCode(Errors.NONE.code())).collect(Collectors.toList()));
|
||||
|
||||
env.kafkaClient().prepareResponse(new AlterUserScramCredentialsResponse(responseData));
|
||||
|
||||
AlterUserScramCredentialsResult result = env.adminClient().alterUserScramCredentials(Arrays.asList(
|
||||
new UserScramCredentialDeletion(user0Name, user0ScramMechanism0),
|
||||
new UserScramCredentialUpsertion(user0Name, new ScramCredentialInfo(user0ScramMechanism1, 8192), "password"),
|
||||
new UserScramCredentialUpsertion(user1Name, new ScramCredentialInfo(user1ScramMechanism0, 8192), "password"),
|
||||
new UserScramCredentialDeletion(user2Name, user2ScramMechanism0)));
|
||||
Map<String, KafkaFuture<Void>> resultData = result.values();
|
||||
assertEquals(3, resultData.size());
|
||||
Arrays.asList(user0Name, user1Name, user2Name).stream().forEach(u -> {
|
||||
assertTrue(resultData.containsKey(u));
|
||||
assertFalse(resultData.get(u).isCompletedExceptionally());
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private void createAlterLogDirsResponse(AdminClientUnitTestEnv env, Node node, Errors error, int... partitions) {
|
||||
env.kafkaClient().prepareResponseFrom(
|
||||
prepareAlterLogDirsResponse(error, "topic", partitions), node);
|
||||
|
|
|
@ -826,6 +826,16 @@ public class MockAdminClient extends AdminClient {
|
|||
throw new UnsupportedOperationException("Not implement yet");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DescribeUserScramCredentialsResult describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options) {
|
||||
throw new UnsupportedOperationException("Not implemented yet");
|
||||
}
|
||||
|
||||
@Override
|
||||
public AlterUserScramCredentialsResult alterUserScramCredentials(List<UserScramCredentialAlteration> alterations, AlterUserScramCredentialsOptions options) {
|
||||
throw new UnsupportedOperationException("Not implemented yet");
|
||||
}
|
||||
|
||||
@Override
|
||||
synchronized public void close(Duration timeout) {}
|
||||
|
||||
|
|
|
@ -26,11 +26,19 @@ public class ProtoUtilsTest {
|
|||
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);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,10 +43,10 @@ public class ScramFormatterTest {
|
|||
String s1 = "r=rOprNGfwEbeRWgbNEkqO%hvYDpWUa2RaTCAfuxFIlj)hNlF$k0,s=W22ZaJ0SNY7soEsUEjb6gQ==,i=4096";
|
||||
String c2 = "c=biws,r=rOprNGfwEbeRWgbNEkqO%hvYDpWUa2RaTCAfuxFIlj)hNlF$k0,p=dHzbZapWIk4jUhN+Ute9ytag9zjfMHgsqmmiz7AndVQ=";
|
||||
String s2 = "v=6rriTRBi23WpRR/wtup+mMhUZUn/dB5nLTJRsjl95G4=";
|
||||
ClientFirstMessage clientFirst = new ClientFirstMessage(formatter.toBytes(c1));
|
||||
ServerFirstMessage serverFirst = new ServerFirstMessage(formatter.toBytes(s1));
|
||||
ClientFinalMessage clientFinal = new ClientFinalMessage(formatter.toBytes(c2));
|
||||
ServerFinalMessage serverFinal = new ServerFinalMessage(formatter.toBytes(s2));
|
||||
ClientFirstMessage clientFirst = new ClientFirstMessage(ScramFormatter.toBytes(c1));
|
||||
ServerFirstMessage serverFirst = new ServerFirstMessage(ScramFormatter.toBytes(s1));
|
||||
ClientFinalMessage clientFinal = new ClientFinalMessage(ScramFormatter.toBytes(c2));
|
||||
ServerFinalMessage serverFinal = new ServerFinalMessage(ScramFormatter.toBytes(s2));
|
||||
|
||||
String username = clientFirst.saslName();
|
||||
assertEquals("user", username);
|
||||
|
@ -82,12 +82,12 @@ public class ScramFormatterTest {
|
|||
String[] usernames = {"user1", "123", "1,2", "user=A", "user==B", "user,1", "user 1", ",", "=", ",=", "=="};
|
||||
ScramFormatter formatter = new ScramFormatter(ScramMechanism.SCRAM_SHA_256);
|
||||
for (String username : usernames) {
|
||||
String saslName = formatter.saslName(username);
|
||||
String saslName = ScramFormatter.saslName(username);
|
||||
// There should be no commas in saslName (comma is used as field separator in SASL messages)
|
||||
assertEquals(-1, saslName.indexOf(','));
|
||||
// There should be no "=" in the saslName apart from those used in encoding (comma is =2C and equals is =3D)
|
||||
assertEquals(-1, saslName.replace("=2C", "").replace("=3D", "").indexOf('='));
|
||||
assertEquals(username, formatter.username(saslName));
|
||||
assertEquals(username, ScramFormatter.username(saslName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -85,13 +85,13 @@ public class ScramMessagesTest {
|
|||
str = String.format("n,,n=test=2Cuser,r=%s", nonce);
|
||||
m = createScramMessage(ClientFirstMessage.class, str);
|
||||
checkClientFirstMessage(m, "test=2Cuser", nonce, "");
|
||||
assertEquals("test,user", formatter.username(m.saslName()));
|
||||
assertEquals("test,user", ScramFormatter.username(m.saslName()));
|
||||
|
||||
// Username containing equals, encoded as =3D
|
||||
str = String.format("n,,n=test=3Duser,r=%s", nonce);
|
||||
m = createScramMessage(ClientFirstMessage.class, str);
|
||||
checkClientFirstMessage(m, "test=3Duser", nonce, "");
|
||||
assertEquals("test=user", formatter.username(m.saslName()));
|
||||
assertEquals("test=user", ScramFormatter.username(m.saslName()));
|
||||
|
||||
// Optional authorization id specified
|
||||
str = String.format("n,a=testauthzid,n=testuser,r=%s", nonce);
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package kafka.admin
|
||||
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.util.{Collections, Properties}
|
||||
|
||||
|
@ -28,8 +29,8 @@ import kafka.server.{ConfigEntityName, ConfigType, Defaults, DynamicBrokerConfig
|
|||
import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit, PasswordEncoder}
|
||||
import kafka.utils.Implicits._
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasOptions, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, DescribeConfigsOptions, ListTopicsOptions, ScramCredentialInfo, UserScramCredentialDeletion, UserScramCredentialUpsertion, Config => JConfig, ScramMechanism => PublicScramMechanism}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasOptions, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, DescribeConfigsOptions, ListTopicsOptions, Config => JConfig}
|
||||
import org.apache.kafka.common.config.ConfigResource
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.errors.InvalidConfigurationException
|
||||
|
@ -56,9 +57,10 @@ import scala.collection._
|
|||
* <li> broker: --broker <broker-id> OR --entity-type brokers --entity-name <broker-id>
|
||||
* <li> broker-logger: --broker-logger <broker-id> OR --entity-type broker-loggers --entity-name <broker-id>
|
||||
* </ul>
|
||||
* --user-defaults, --client-defaults, or --broker-defaults may be when describing or altering default configuration for users,
|
||||
* clients, and brokers, respectively. Alternatively, --entity-default may be used instead of --entity-name.
|
||||
*
|
||||
* --entity-type <users|clients|brokers> --entity-default may be specified in place of --entity-type <users|clients|brokers> --entity-name <entityName>
|
||||
* when describing or altering default configuration for users, clients, or brokers, respectively.
|
||||
* Alternatively, --user-defaults, --client-defaults, or --broker-defaults may be specified in place of
|
||||
* --entity-type <users|clients|brokers> --entity-default, respectively.
|
||||
*/
|
||||
object ConfigCommand extends Config {
|
||||
|
||||
|
@ -310,7 +312,7 @@ object ConfigCommand extends Config {
|
|||
val entityNames = opts.entityNames
|
||||
val entityTypeHead = entityTypes.head
|
||||
val entityNameHead = entityNames.head
|
||||
val configsToBeAddedMap = parseConfigsToBeAdded(opts).asScala
|
||||
val configsToBeAddedMap = parseConfigsToBeAdded(opts).asScala.toMap // no need for mutability
|
||||
val configsToBeAdded = configsToBeAddedMap.map { case (k, v) => (k, new ConfigEntry(k, v)) }
|
||||
val configsToBeDeleted = parseConfigsToBeDeleted(opts)
|
||||
|
||||
|
@ -365,45 +367,40 @@ 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)
|
||||
|
||||
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)
|
||||
if (hasQuotaConfigsToAdd || hasQuotaConfigsToDelete) {
|
||||
alterQuotaConfigs(adminClient, entityTypes, entityNames, configsToBeAddedMap, configsToBeDeleted)
|
||||
} else {
|
||||
// handle altering user SCRAM credential configs
|
||||
if (entityNames.size != 1)
|
||||
// should never happen, if we get here then it is a bug
|
||||
throw new IllegalStateException(s"Altering user SCRAM credentials should never occur for more zero or multiple users: $entityNames")
|
||||
alterUserScramCredentialConfigs(adminClient, entityNames.head, scramConfigsToAddMap, scramConfigsToDelete)
|
||||
}
|
||||
|
||||
case _ => throw new IllegalArgumentException(s"Unsupported entity type: $entityTypeHead")
|
||||
}
|
||||
|
@ -414,6 +411,65 @@ object ConfigCommand extends Config {
|
|||
println(s"Completed updating default config for $entityTypeHead in the cluster.")
|
||||
}
|
||||
|
||||
private def alterUserScramCredentialConfigs(adminClient: Admin, user: String, scramConfigsToAddMap: Map[String, ConfigEntry], scramConfigsToDelete: Seq[String]) = {
|
||||
val deletions = scramConfigsToDelete.map(mechanismName =>
|
||||
new UserScramCredentialDeletion(user, PublicScramMechanism.fromMechanismName(mechanismName)))
|
||||
|
||||
def iterationsAndPasswordBytes(mechanism: ScramMechanism, credentialStr: String): (Integer, Array[Byte]) = {
|
||||
val pattern = "(?:iterations=(\\-?[0-9]*),)?password=(.*)".r
|
||||
val (iterations, password) = credentialStr match {
|
||||
case pattern(iterations, password) => (if (iterations != null && iterations != "-1") iterations.toInt else DefaultScramIterations, password)
|
||||
case _ => throw new IllegalArgumentException(s"Invalid credential property $mechanism=$credentialStr")
|
||||
}
|
||||
if (iterations < mechanism.minIterations)
|
||||
throw new IllegalArgumentException(s"Iterations $iterations is less than the minimum ${mechanism.minIterations} required for ${mechanism.mechanismName}")
|
||||
(iterations, password.getBytes(StandardCharsets.UTF_8))
|
||||
}
|
||||
|
||||
val upsertions = scramConfigsToAddMap.map { case (mechanismName, configEntry) =>
|
||||
val (iterations, passwordBytes) = iterationsAndPasswordBytes(ScramMechanism.forMechanismName(mechanismName), configEntry.value)
|
||||
new UserScramCredentialUpsertion(user, new ScramCredentialInfo(PublicScramMechanism.fromMechanismName(mechanismName), iterations), passwordBytes)
|
||||
}
|
||||
// we are altering only a single user by definition, so we don't have to worry about one user succeeding and another
|
||||
// failing; therefore just check the success of all the futures (since there will only be 1)
|
||||
adminClient.alterUserScramCredentials((deletions ++ upsertions).toList.asJava).all.get(60, TimeUnit.SECONDS)
|
||||
}
|
||||
|
||||
private def alterQuotaConfigs(adminClient: Admin, entityTypes: List[String], entityNames: List[String], configsToBeAddedMap: Map[String, String], configsToBeDeleted: Seq[String]) = {
|
||||
// handle altering client/user quota configs
|
||||
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 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)
|
||||
}
|
||||
|
||||
private[admin] def describeConfig(adminClient: Admin, opts: ConfigCommandOptions): Unit = {
|
||||
val entityTypes = opts.entityTypes
|
||||
val entityNames = opts.entityNames
|
||||
|
@ -423,7 +479,7 @@ object ConfigCommand extends Config {
|
|||
case ConfigType.Topic | ConfigType.Broker | BrokerLoggerConfigType =>
|
||||
describeResourceConfig(adminClient, entityTypes.head, entityNames.headOption, describeAll)
|
||||
case ConfigType.User | ConfigType.Client =>
|
||||
describeClientQuotasConfig(adminClient, entityTypes, entityNames)
|
||||
describeClientQuotaAndUserScramCredentialConfigs(adminClient, entityTypes, entityNames)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -492,8 +548,9 @@ object ConfigCommand extends Config {
|
|||
}).toSeq
|
||||
}
|
||||
|
||||
private def describeClientQuotasConfig(adminClient: Admin, entityTypes: List[String], entityNames: List[String]) = {
|
||||
getAllClientQuotasConfigs(adminClient, entityTypes, entityNames).foreach { case (entity, entries) =>
|
||||
private def describeClientQuotaAndUserScramCredentialConfigs(adminClient: Admin, entityTypes: List[String], entityNames: List[String]) = {
|
||||
val quotaConfigs = getAllClientQuotasConfigs(adminClient, entityTypes, entityNames)
|
||||
quotaConfigs.foreach { case (entity, entries) =>
|
||||
val entityEntries = entity.entries.asScala
|
||||
|
||||
def entitySubstr(entityType: String): Option[String] =
|
||||
|
@ -508,7 +565,21 @@ 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("")) {
|
||||
val result = adminClient.describeUserScramCredentials(entityNames.asJava)
|
||||
result.users.get(30, TimeUnit.SECONDS).asScala.foreach(user => {
|
||||
try {
|
||||
val description = result.description(user).get(30, TimeUnit.SECONDS)
|
||||
val descriptionText = description.credentialInfos.asScala.map(info => s"${info.mechanism.mechanismName}=iterations=${info.iterations}").mkString(", ")
|
||||
println(s"SCRAM credential configs for user-principal '$user' are $descriptionText")
|
||||
} catch {
|
||||
case e: Exception => println(s"Error retrieving SCRAM credential configs for user-principal '$user': ${e.getClass.getSimpleName}: ${e.getMessage}")
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -659,7 +730,7 @@ object ConfigCommand extends Config {
|
|||
.ofType(classOf[String])
|
||||
val alterOpt = parser.accepts("alter", "Alter the configuration for the entity.")
|
||||
val describeOpt = parser.accepts("describe", "List configs for the given entity.")
|
||||
val allOpt = parser.accepts("all", "List all configs for the given entity (includes static configuration when the entity type is brokers)")
|
||||
val allOpt = parser.accepts("all", "List all configs for the given topic, broker, or broker-logger entity (includes static configuration when the entity type is brokers)")
|
||||
|
||||
val entityType = parser.accepts("entity-type", "Type of entity (topics/clients/users/brokers/broker-loggers)")
|
||||
.withRequiredArg
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import java.util
|
||||
import java.util.{Collections, Properties}
|
||||
|
||||
import kafka.admin.{AdminOperationException, AdminUtils}
|
||||
|
@ -26,19 +27,22 @@ import kafka.metrics.KafkaMetricsGroup
|
|||
import kafka.server.DynamicConfig.QuotaConfigs
|
||||
import kafka.utils._
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.clients.admin.AlterConfigOp
|
||||
import org.apache.kafka.clients.admin.{AlterConfigOp, ScramMechanism}
|
||||
import org.apache.kafka.clients.admin.AlterConfigOp.OpType
|
||||
import org.apache.kafka.common.config.ConfigDef.ConfigKey
|
||||
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, LogLevelConfig}
|
||||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException
|
||||
import org.apache.kafka.common.errors.{ApiException, InvalidConfigurationException, InvalidPartitionsException, InvalidReplicaAssignmentException, InvalidRequestException, ReassignmentInProgressException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedVersionException}
|
||||
import org.apache.kafka.common.internals.Topic
|
||||
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
|
||||
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic
|
||||
import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicConfigs, CreatableTopicResult}
|
||||
import org.apache.kafka.common.message.DescribeConfigsResponseData
|
||||
import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, AlterUserScramCredentialsResponseData, DescribeConfigsResponseData, DescribeUserScramCredentialsResponseData}
|
||||
import org.apache.kafka.common.message.DescribeConfigsRequestData.DescribeConfigsResource
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.security.scram.internals.{ScramMechanism => InternalScramMechanism}
|
||||
import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy}
|
||||
import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
|
@ -46,6 +50,7 @@ import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity,
|
|||
import org.apache.kafka.common.requests.CreateTopicsRequest._
|
||||
import org.apache.kafka.common.requests.DescribeConfigsResponse.ConfigSource
|
||||
import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError, DescribeConfigsResponse}
|
||||
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter}
|
||||
import org.apache.kafka.common.utils.Sanitizer
|
||||
|
||||
import scala.collection.{Map, mutable, _}
|
||||
|
@ -980,4 +985,248 @@ class AdminManager(val config: KafkaConfig,
|
|||
entry.entity -> apiError
|
||||
}.toMap
|
||||
}
|
||||
|
||||
private val usernameMustNotBeEmptyMsg = "Username must not be empty"
|
||||
private val errorProcessingDescribe = "Error processing describe user SCRAM credential configs request"
|
||||
private val attemptToDescribeUserThatDoesNotExist = "Attempt to describe a user credential that does not exist"
|
||||
|
||||
def describeUserScramCredentials(users: Option[Seq[String]]): DescribeUserScramCredentialsResponseData = {
|
||||
val describingAllUsers = !users.isDefined || users.get.isEmpty
|
||||
val retval = new DescribeUserScramCredentialsResponseData()
|
||||
val userResults = mutable.Map[String, DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult]()
|
||||
|
||||
def addToResultsIfHasScramCredential(user: String, userConfig: Properties, explicitUser: Boolean = false): Unit = {
|
||||
val result = new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult().setUser(user)
|
||||
val configKeys = userConfig.stringPropertyNames
|
||||
val hasScramCredential = ScramMechanism.values().toList.exists(key => key != ScramMechanism.UNKNOWN && configKeys.contains(key.mechanismName))
|
||||
if (hasScramCredential) {
|
||||
val credentialInfos = new util.ArrayList[CredentialInfo]
|
||||
try {
|
||||
ScramMechanism.values().filter(_ != ScramMechanism.UNKNOWN).foreach { mechanism =>
|
||||
val propertyValue = userConfig.getProperty(mechanism.mechanismName)
|
||||
if (propertyValue != null) {
|
||||
val iterations = ScramCredentialUtils.credentialFromString(propertyValue).iterations
|
||||
credentialInfos.add(new CredentialInfo().setMechanism(mechanism.`type`).setIterations(iterations))
|
||||
}
|
||||
}
|
||||
result.setCredentialInfos(credentialInfos)
|
||||
} catch {
|
||||
case e: Exception => { // should generally never happen, but just in case bad data gets in...
|
||||
val apiError = apiErrorFrom(e, errorProcessingDescribe)
|
||||
result.setErrorCode(apiError.error.code).setErrorMessage(apiError.error.message)
|
||||
}
|
||||
}
|
||||
userResults += (user -> result)
|
||||
} else if (explicitUser) {
|
||||
// it is an error to request credentials for a user that has no credentials
|
||||
result.setErrorCode(Errors.RESOURCE_NOT_FOUND.code).setErrorMessage(s"$attemptToDescribeUserThatDoesNotExist: $user")
|
||||
userResults += (user -> result)
|
||||
}
|
||||
}
|
||||
|
||||
def collectRetrievedResults(): Unit = {
|
||||
if (describingAllUsers) {
|
||||
val usersSorted = SortedSet.empty[String] ++ userResults.keys
|
||||
usersSorted.foreach { user => retval.results.add(userResults(user)) }
|
||||
} else {
|
||||
// be sure to only include a single copy of a result for any user requested multiple times
|
||||
users.get.distinct.foreach { user => retval.results.add(userResults(user)) }
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
if (describingAllUsers)
|
||||
adminZkClient.fetchAllEntityConfigs(ConfigType.User).foreach {
|
||||
case (user, properties) => addToResultsIfHasScramCredential(user, properties) }
|
||||
else {
|
||||
// describing specific users
|
||||
val illegalUsers = users.get.filter(_.isEmpty).toSet
|
||||
illegalUsers.foreach { user =>
|
||||
userResults += (user -> new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(Errors.RESOURCE_NOT_FOUND.code)
|
||||
.setErrorMessage(usernameMustNotBeEmptyMsg)) }
|
||||
val duplicatedUsers = users.get.groupBy(identity).filter(
|
||||
userAndOccurrencesTuple => userAndOccurrencesTuple._2.length > 1).keys
|
||||
duplicatedUsers.filterNot(illegalUsers.contains).foreach { user =>
|
||||
userResults += (user -> new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(Errors.DUPLICATE_RESOURCE.code)
|
||||
.setErrorMessage(s"Cannot describe SCRAM credentials for the same user twice in a single request: $user")) }
|
||||
val usersToSkip = illegalUsers ++ duplicatedUsers
|
||||
users.get.filterNot(usersToSkip.contains).foreach { user =>
|
||||
try {
|
||||
val userConfigs = adminZkClient.fetchEntityConfig(ConfigType.User, Sanitizer.sanitize(user))
|
||||
addToResultsIfHasScramCredential(user, userConfigs, true)
|
||||
} catch {
|
||||
case e: Exception => {
|
||||
val apiError = apiErrorFrom(e, errorProcessingDescribe)
|
||||
userResults += (user -> new DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(apiError.error.code)
|
||||
.setErrorMessage(apiError.error.message))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
collectRetrievedResults()
|
||||
} catch {
|
||||
case e: Exception => {
|
||||
// this should generally only happen when we get a failure trying to retrieve all user configs from ZooKeeper
|
||||
val apiError = apiErrorFrom(e, errorProcessingDescribe)
|
||||
retval.setErrorCode(apiError.error.code).setErrorMessage(apiError.messageWithFallback())
|
||||
}
|
||||
}
|
||||
retval
|
||||
}
|
||||
|
||||
def apiErrorFrom(e: Exception, message: String): ApiError = {
|
||||
if (e.isInstanceOf[ApiException])
|
||||
info(message, e)
|
||||
else
|
||||
error(message, e)
|
||||
ApiError.fromThrowable(e)
|
||||
}
|
||||
|
||||
case class requestStatus(user: String, mechanism: Option[ScramMechanism], legalRequest: Boolean, iterations: Int) {}
|
||||
|
||||
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).mechanismName
|
||||
}
|
||||
|
||||
val retval = new AlterUserScramCredentialsResponseData()
|
||||
|
||||
// fail any user that is invalid due to an empty user name, an unknown SCRAM mechanism, or unacceptable number of iterations
|
||||
val maxIterations = 16384
|
||||
val illegalUpsertions = upsertions.map(upsertion =>
|
||||
if (upsertion.name.isEmpty)
|
||||
requestStatus(upsertion.name, None, false, upsertion.iterations) // no determined mechanism -- empty user is the cause of failure
|
||||
else {
|
||||
val publicScramMechanism = scramMechanism(upsertion.mechanism)
|
||||
if (publicScramMechanism == ScramMechanism.UNKNOWN) {
|
||||
requestStatus(upsertion.name, Some(publicScramMechanism), false, upsertion.iterations) // unknown mechanism is the cause of failure
|
||||
} else {
|
||||
if (upsertion.iterations < InternalScramMechanism.forMechanismName(publicScramMechanism.mechanismName).minIterations
|
||||
|| upsertion.iterations > maxIterations) {
|
||||
requestStatus(upsertion.name, Some(publicScramMechanism), false, upsertion.iterations) // known mechanism, bad iterations is the cause of failure
|
||||
} else {
|
||||
requestStatus(upsertion.name, Some(publicScramMechanism), true, upsertion.iterations) // legal
|
||||
}
|
||||
}
|
||||
}).filter { !_.legalRequest }
|
||||
val illegalDeletions = deletions.map(deletion =>
|
||||
if (deletion.name.isEmpty) {
|
||||
requestStatus(deletion.name, None, false, 0) // no determined mechanism -- empty user is the cause of failure
|
||||
} else {
|
||||
val publicScramMechanism = scramMechanism(deletion.mechanism)
|
||||
requestStatus(deletion.name, Some(publicScramMechanism), publicScramMechanism != ScramMechanism.UNKNOWN, 0)
|
||||
}).filter { !_.legalRequest }
|
||||
// map user names to error messages
|
||||
val unknownScramMechanismMsg = "Unknown SCRAM mechanism"
|
||||
val tooFewIterationsMsg = "Too few iterations"
|
||||
val tooManyIterationsMsg = "Too many iterations"
|
||||
val illegalRequestsByUser =
|
||||
illegalDeletions.map(requestStatus =>
|
||||
if (requestStatus.user.isEmpty) {
|
||||
(requestStatus.user, usernameMustNotBeEmptyMsg)
|
||||
} else {
|
||||
(requestStatus.user, unknownScramMechanismMsg)
|
||||
}
|
||||
).toMap ++ illegalUpsertions.map(requestStatus =>
|
||||
if (requestStatus.user.isEmpty) {
|
||||
(requestStatus.user, usernameMustNotBeEmptyMsg)
|
||||
} else if (requestStatus.mechanism == Some(ScramMechanism.UNKNOWN)) {
|
||||
(requestStatus.user, unknownScramMechanismMsg)
|
||||
} else {
|
||||
(requestStatus.user, if (requestStatus.iterations > maxIterations) {tooManyIterationsMsg} else {tooFewIterationsMsg})
|
||||
}
|
||||
).toMap
|
||||
|
||||
illegalRequestsByUser.foreach { case (user, errorMessage) =>
|
||||
retval.results.add(new AlterUserScramCredentialsResult().setUser(user)
|
||||
.setErrorCode(if (errorMessage == unknownScramMechanismMsg) {Errors.UNSUPPORTED_SASL_MECHANISM.code} else {Errors.UNACCEPTABLE_CREDENTIAL.code})
|
||||
.setErrorMessage(errorMessage)) }
|
||||
|
||||
val invalidUsers = (illegalUpsertions ++ illegalDeletions).map(_.user).toSet
|
||||
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)))
|
||||
|
||||
val usersWithDuplicateUserMechanismPairs = initiallyValidUserMechanismPairs.groupBy(identity).filter (
|
||||
userMechanismPairAndOccurrencesTuple => userMechanismPairAndOccurrencesTuple._2.length > 1).keys.map(userMechanismPair => userMechanismPair._1).toSet
|
||||
usersWithDuplicateUserMechanismPairs.foreach { user =>
|
||||
retval.results.add(new AlterUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(Errors.DUPLICATE_RESOURCE.code).setErrorMessage("A user credential cannot be altered twice in the same request")) }
|
||||
|
||||
def potentiallyValidUserMechanismPairs = initiallyValidUserMechanismPairs.filter(pair => !usersWithDuplicateUserMechanismPairs.contains(pair._1))
|
||||
|
||||
val potentiallyValidUsers = potentiallyValidUserMechanismPairs.map(_._1).toSet
|
||||
val configsByPotentiallyValidUser = potentiallyValidUsers.map(user => (user, adminZkClient.fetchEntityConfig(ConfigType.User, Sanitizer.sanitize(user)))).toMap
|
||||
|
||||
// check for deletion of a credential that does not exist
|
||||
val invalidDeletions = deletions.filter(deletion => potentiallyValidUsers.contains(deletion.name)).filter(deletion =>
|
||||
configsByPotentiallyValidUser(deletion.name).getProperty(mechanismName(deletion.mechanism)) == null)
|
||||
val invalidUsersDueToInvalidDeletions = invalidDeletions.map(_.name).toSet
|
||||
invalidUsersDueToInvalidDeletions.foreach { user =>
|
||||
retval.results.add(new AlterUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(Errors.RESOURCE_NOT_FOUND.code).setErrorMessage("Attempt to delete a user credential that does not exist")) }
|
||||
|
||||
// now prepare the new set of property values for users that don't have any issues identified above,
|
||||
// keeping track of ones that fail
|
||||
val usersToTryToAlter = potentiallyValidUsers.diff(invalidUsersDueToInvalidDeletions)
|
||||
val usersFailedToPrepareProperties = usersToTryToAlter.map(user => {
|
||||
try {
|
||||
// deletions: remove property keys
|
||||
deletions.filter(deletion => usersToTryToAlter.contains(deletion.name)).foreach { deletion =>
|
||||
configsByPotentiallyValidUser(deletion.name).remove(mechanismName(deletion.mechanism)) }
|
||||
// upsertions: put property key/value
|
||||
upsertions.filter(upsertion => usersToTryToAlter.contains(upsertion.name)).foreach { upsertion =>
|
||||
val mechanism = InternalScramMechanism.forMechanismName(mechanismName(upsertion.mechanism))
|
||||
val credential = new ScramFormatter(mechanism)
|
||||
.generateCredential(upsertion.salt, upsertion.saltedPassword, upsertion.iterations)
|
||||
configsByPotentiallyValidUser(upsertion.name).put(mechanismName(upsertion.mechanism), ScramCredentialUtils.credentialToString(credential)) }
|
||||
(user) // success, 1 element, won't be matched
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
info(s"Error encountered while altering user SCRAM credentials", e)
|
||||
(user, e) // fail, 2 elements, will be matched
|
||||
}
|
||||
}).collect { case (user: String, exception: Exception) => (user, exception) }.toMap
|
||||
|
||||
// now persist the properties we have prepared, again keeping track of whatever fails
|
||||
val usersFailedToPersist = usersToTryToAlter.filterNot(usersFailedToPrepareProperties.contains).map(user => {
|
||||
try {
|
||||
adminZkClient.changeConfigs(ConfigType.User, Sanitizer.sanitize(user), configsByPotentiallyValidUser(user))
|
||||
(user) // success, 1 element, won't be matched
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
info(s"Error encountered while altering user SCRAM credentials", e)
|
||||
(user, e) // fail, 2 elements, will be matched
|
||||
}
|
||||
}).collect { case (user: String, exception: Exception) => (user, exception) }.toMap
|
||||
|
||||
// report failures
|
||||
usersFailedToPrepareProperties.++(usersFailedToPersist).foreach { case (user, exception) =>
|
||||
val error = Errors.forException(exception)
|
||||
retval.results.add(new AlterUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(error.code)
|
||||
.setErrorMessage(error.message)) }
|
||||
|
||||
// report successes
|
||||
usersToTryToAlter.filterNot(usersFailedToPrepareProperties.contains).filterNot(usersFailedToPersist.contains).foreach { user =>
|
||||
retval.results.add(new AlterUserScramCredentialsResult()
|
||||
.setUser(user)
|
||||
.setErrorCode(Errors.NONE.code)) }
|
||||
|
||||
retval
|
||||
}
|
||||
}
|
||||
|
|
|
@ -179,6 +179,8 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
case ApiKeys.OFFSET_DELETE => handleOffsetDeleteRequest(request)
|
||||
case ApiKeys.DESCRIBE_CLIENT_QUOTAS => handleDescribeClientQuotasRequest(request)
|
||||
case ApiKeys.ALTER_CLIENT_QUOTAS => handleAlterClientQuotasRequest(request)
|
||||
case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => handleDescribeUserScramCredentialsRequest(request)
|
||||
case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => handleAlterUserScramCredentialsRequest(request)
|
||||
}
|
||||
} catch {
|
||||
case e: FatalExitError => throw e
|
||||
|
@ -1417,7 +1419,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val states = if (listGroupsRequest.data.statesFilter == null)
|
||||
// Handle a null array the same as empty
|
||||
immutable.Set[String]()
|
||||
else
|
||||
else
|
||||
listGroupsRequest.data.statesFilter.asScala.toSet
|
||||
|
||||
def createResponse(throttleMs: Int, groups: List[GroupOverview], error: Errors): AbstractResponse = {
|
||||
|
@ -3008,6 +3010,37 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
}
|
||||
|
||||
def handleDescribeUserScramCredentialsRequest(request: RequestChannel.Request): Unit = {
|
||||
val describeUserScramCredentialsRequest = request.body[DescribeUserScramCredentialsRequest]
|
||||
|
||||
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)) {
|
||||
val result = adminManager.alterUserScramCredentials(
|
||||
alterUserScramCredentialsRequest.data.upsertions().asScala, alterUserScramCredentialsRequest.data.deletions().asScala)
|
||||
sendResponseMaybeThrottle(request, requestThrottleMs =>
|
||||
new AlterUserScramCredentialsResponse(result.setThrottleTimeMs(requestThrottleMs)))
|
||||
} else {
|
||||
sendResponseMaybeThrottle(request, requestThrottleMs =>
|
||||
alterUserScramCredentialsRequest.getErrorResponse(requestThrottleMs, Errors.CLUSTER_AUTHORIZATION_FAILED.exception))
|
||||
}
|
||||
}
|
||||
|
||||
// private package for testing
|
||||
private[server] def authorize(requestContext: RequestContext,
|
||||
operation: AclOperation,
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.kafka.common.{Cluster, Reconfigurable}
|
|||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.security.auth._
|
||||
import org.apache.kafka.common.security.scram.ScramCredential
|
||||
import org.apache.kafka.server.quota._
|
||||
import org.junit.Assert._
|
||||
import org.junit.{After, Before, Test}
|
||||
|
@ -204,14 +203,19 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup {
|
|||
}
|
||||
}
|
||||
|
||||
private def addUser(user: String, leader: Int): GroupedUser = {
|
||||
val password = s"$user:secret"
|
||||
createScramCredentials(zkConnect, user, password)
|
||||
servers.foreach { server =>
|
||||
val cache = server.credentialProvider.credentialCache.cache(kafkaClientSaslMechanism, classOf[ScramCredential])
|
||||
TestUtils.waitUntilTrue(() => cache.get(user) != null, "SCRAM credentials not created")
|
||||
}
|
||||
private def passwordForUser(user: String) = {
|
||||
s"$user:secret"
|
||||
}
|
||||
|
||||
private def addUser(user: String, leader: Int): GroupedUser = {
|
||||
val adminClient = createAdminClient()
|
||||
createScramCredentials(adminClient, user, passwordForUser(user))
|
||||
waitForUserScramCredentialToAppearOnAllBrokers(user, kafkaClientSaslMechanism)
|
||||
groupedUser(adminClient, user, leader)
|
||||
}
|
||||
|
||||
private def groupedUser(adminClient: Admin, user: String, leader: Int): GroupedUser = {
|
||||
val password = passwordForUser(user)
|
||||
val userGroup = group(user)
|
||||
val topic = s"${userGroup}_topic"
|
||||
val producerClientId = s"$user:producer-client-id"
|
||||
|
@ -226,7 +230,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup {
|
|||
consumerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, ScramLoginModule(user, password).toString)
|
||||
|
||||
GroupedUser(user, userGroup, topic, servers(leader), producerClientId, consumerClientId,
|
||||
createProducer(), createConsumer(), createAdminClient())
|
||||
createProducer(), createConsumer(), adminClient)
|
||||
}
|
||||
|
||||
case class GroupedUser(user: String, userGroup: String, topic: String, leaderNode: KafkaServer,
|
||||
|
|
|
@ -16,18 +16,18 @@
|
|||
*/
|
||||
package kafka.api
|
||||
|
||||
import java.util
|
||||
import java.util.Properties
|
||||
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.{JaasTestUtils, TestUtils}
|
||||
import kafka.zk.ConfigEntityChangeNotificationZNode
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, ScramCredentialInfo, UserScramCredentialAlteration, UserScramCredentialUpsertion, ScramMechanism => PublicScramMechanism}
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
|
||||
import org.apache.kafka.common.security.scram.ScramCredential
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMechanism
|
||||
import org.apache.kafka.common.security.token.delegation.DelegationToken
|
||||
import org.junit.Before
|
||||
import org.junit.Assert._
|
||||
import org.junit.{Before, Test}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
|
@ -35,7 +35,9 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest
|
|||
|
||||
val kafkaClientSaslMechanism = "SCRAM-SHA-256"
|
||||
val kafkaServerSaslMechanisms = ScramMechanism.mechanismNames.asScala.toList
|
||||
|
||||
override protected def securityProtocol = SecurityProtocol.SASL_SSL
|
||||
|
||||
override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism))
|
||||
override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism))
|
||||
|
||||
|
@ -45,6 +47,8 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest
|
|||
override val kafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, JaasTestUtils.KafkaScramAdmin)
|
||||
private val kafkaPassword = JaasTestUtils.KafkaScramAdminPassword
|
||||
|
||||
private val privilegedAdminClientConfig = new Properties()
|
||||
|
||||
this.serverConfig.setProperty(KafkaConfig.DelegationTokenMasterKeyProp, "testKey")
|
||||
|
||||
override def configureSecurityBeforeServersStart(): Unit = {
|
||||
|
@ -54,27 +58,43 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest
|
|||
createScramCredentials(zkConnect, kafkaPrincipal.getName, kafkaPassword)
|
||||
}
|
||||
|
||||
override def createPrivilegedAdminClient() = createScramAdminClient(kafkaClientSaslMechanism, kafkaPrincipal.getName, kafkaPassword)
|
||||
|
||||
override def configureSecurityAfterServersStart(): Unit = {
|
||||
super.configureSecurityAfterServersStart()
|
||||
|
||||
// create scram credential for user "scram-user"
|
||||
createScramCredentials(zkConnect, clientPrincipal.getName, clientPassword)
|
||||
waitForScramCredentials(clientPrincipal.getName)
|
||||
createScramCredentialsViaPrivilegedAdminClient(clientPrincipal.getName, clientPassword)
|
||||
waitForUserScramCredentialToAppearOnAllBrokers(clientPrincipal.getName, kafkaClientSaslMechanism)
|
||||
|
||||
//create a token with "scram-user" credentials
|
||||
val token = createDelegationToken()
|
||||
//create a token with "scram-user" credentials and a privileged token with scram-admin credentials
|
||||
val tokens = createDelegationTokens()
|
||||
val token = tokens._1
|
||||
val privilegedToken = tokens._2
|
||||
|
||||
privilegedAdminClientConfig.putAll(adminClientConfig)
|
||||
|
||||
// pass token to client jaas config
|
||||
val clientLoginContext = JaasTestUtils.tokenClientLoginModule(token.tokenInfo().tokenId(), token.hmacAsBase64String())
|
||||
producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext)
|
||||
consumerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext)
|
||||
adminClientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext)
|
||||
val privilegedClientLoginContext = JaasTestUtils.tokenClientLoginModule(privilegedToken.tokenInfo().tokenId(), privilegedToken.hmacAsBase64String())
|
||||
privilegedAdminClientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, privilegedClientLoginContext)
|
||||
}
|
||||
|
||||
private def waitForScramCredentials(clientPrincipal: String): Unit = {
|
||||
servers.foreach { server =>
|
||||
val cache = server.credentialProvider.credentialCache.cache(kafkaClientSaslMechanism, classOf[ScramCredential])
|
||||
TestUtils.waitUntilTrue(() => cache.get(clientPrincipal) != null, s"SCRAM credentials not created for $clientPrincipal")
|
||||
@Test
|
||||
def testCreateUserWithDelegationToken(): Unit = {
|
||||
val privilegedAdminClient = Admin.create(privilegedAdminClientConfig)
|
||||
try {
|
||||
val user = "user"
|
||||
val results = privilegedAdminClient.alterUserScramCredentials(List[UserScramCredentialAlteration](
|
||||
new UserScramCredentialUpsertion(user, new ScramCredentialInfo(PublicScramMechanism.SCRAM_SHA_256, 4096), "password")).asJava)
|
||||
assertEquals(1, results.values.size)
|
||||
val future = results.values.get(user)
|
||||
future.get // make sure we haven't completed exceptionally
|
||||
} finally {
|
||||
privilegedAdminClient.close()
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -82,24 +102,23 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest
|
|||
override def setUp(): Unit = {
|
||||
startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism), Both))
|
||||
super.setUp()
|
||||
privilegedAdminClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
|
||||
}
|
||||
|
||||
private def createDelegationToken(): DelegationToken = {
|
||||
val config = new util.HashMap[String, Object]
|
||||
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
|
||||
val securityProps: util.Map[Object, Object] =
|
||||
TestUtils.adminClientSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties)
|
||||
securityProps.forEach { (key, value) => config.put(key.asInstanceOf[String], value) }
|
||||
val clientLoginContext = jaasClientLoginModule(kafkaClientSaslMechanism)
|
||||
config.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext)
|
||||
|
||||
val adminClient = Admin.create(config)
|
||||
private def createDelegationTokens(): (DelegationToken, DelegationToken) = {
|
||||
val adminClient = createScramAdminClient(kafkaClientSaslMechanism, clientPrincipal.getName, clientPassword)
|
||||
try {
|
||||
val token = adminClient.createDelegationToken().delegationToken().get()
|
||||
//wait for token to reach all the brokers
|
||||
TestUtils.waitUntilTrue(() => servers.forall(server => !server.tokenCache.tokens().isEmpty),
|
||||
"Timed out waiting for token to propagate to all servers")
|
||||
token
|
||||
val privilegedAdminClient = createScramAdminClient(kafkaClientSaslMechanism, kafkaPrincipal.getName, kafkaPassword)
|
||||
try {
|
||||
val token = adminClient.createDelegationToken().delegationToken().get()
|
||||
val privilegedToken = privilegedAdminClient.createDelegationToken().delegationToken().get()
|
||||
//wait for tokens to reach all the brokers
|
||||
TestUtils.waitUntilTrue(() => servers.forall(server => server.tokenCache.tokens().size() == 2),
|
||||
"Timed out waiting for token to propagate to all servers")
|
||||
(token, privilegedToken)
|
||||
} finally {
|
||||
privilegedAdminClient.close()
|
||||
}
|
||||
} finally {
|
||||
adminClient.close()
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import kafka.security.authorizer.AclAuthorizer
|
|||
import kafka.security.authorizer.AclEntry.WildcardHost
|
||||
import kafka.server._
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.clients.admin.Admin
|
||||
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords}
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
||||
import org.apache.kafka.common.acl._
|
||||
|
@ -68,6 +69,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
|
|||
|
||||
override def configureSecurityBeforeServersStart(): Unit = {
|
||||
AclCommand.main(clusterActionArgs)
|
||||
AclCommand.main(clusterAlterArgs)
|
||||
AclCommand.main(topicBrokerReadAclArgs)
|
||||
}
|
||||
|
||||
|
@ -101,6 +103,14 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
|
|||
s"--cluster",
|
||||
s"--operation=ClusterAction",
|
||||
s"--allow-principal=$kafkaPrincipal")
|
||||
// necessary to create SCRAM credentials via the admin client using the broker's credentials
|
||||
// without this we would need to create the SCRAM credentials via ZooKeeper
|
||||
def clusterAlterArgs: Array[String] = Array("--authorizer-properties",
|
||||
s"zookeeper.connect=$zkConnect",
|
||||
s"--add",
|
||||
s"--cluster",
|
||||
s"--operation=Alter",
|
||||
s"--allow-principal=$kafkaPrincipal")
|
||||
def topicBrokerReadAclArgs: Array[String] = Array("--authorizer-properties",
|
||||
s"zookeeper.connect=$zkConnect",
|
||||
s"--add",
|
||||
|
@ -164,7 +174,8 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
|
|||
s"--producer",
|
||||
s"--allow-principal=$clientPrincipal")
|
||||
|
||||
def ClusterActionAcl = Set(new AccessControlEntry(kafkaPrincipal.toString, WildcardHost, CLUSTER_ACTION, ALLOW))
|
||||
def ClusterActionAndClusterAlterAcls = Set(new AccessControlEntry(kafkaPrincipal.toString, WildcardHost, CLUSTER_ACTION, ALLOW),
|
||||
new AccessControlEntry(kafkaPrincipal.toString, WildcardHost, ALTER, ALLOW))
|
||||
def TopicBrokerReadAcl = Set(new AccessControlEntry(kafkaPrincipal.toString, WildcardHost, READ, ALLOW))
|
||||
def GroupReadAcl = Set(new AccessControlEntry(clientPrincipal.toString, WildcardHost, READ, ALLOW))
|
||||
def TopicReadAcl = Set(new AccessControlEntry(clientPrincipal.toString, WildcardHost, READ, ALLOW))
|
||||
|
@ -191,7 +202,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
|
|||
override def setUp(): Unit = {
|
||||
super.setUp()
|
||||
servers.foreach { s =>
|
||||
TestUtils.waitAndVerifyAcls(ClusterActionAcl, s.dataPlaneRequestProcessor.authorizer.get, clusterResource)
|
||||
TestUtils.waitAndVerifyAcls(ClusterActionAndClusterAlterAcls, s.dataPlaneRequestProcessor.authorizer.get, clusterResource)
|
||||
TestUtils.waitAndVerifyAcls(TopicBrokerReadAcl, s.dataPlaneRequestProcessor.authorizer.get, new ResourcePattern(TOPIC, "*", LITERAL))
|
||||
}
|
||||
// create the test topic with all the brokers as replicas
|
||||
|
@ -545,6 +556,11 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
|
|||
}
|
||||
}
|
||||
|
||||
protected def createScramAdminClient(scramMechanism: String, user: String, password: String): Admin = {
|
||||
createAdminClient(brokerList, securityProtocol, trustStoreFile, clientSaslProperties,
|
||||
scramMechanism, user, password)
|
||||
}
|
||||
|
||||
// Consume records, ignoring at most one TopicAuthorization exception from previously sent request
|
||||
private def consumeRecordsIgnoreOneAuthorizationException(consumer: Consumer[Array[Byte], Array[Byte]],
|
||||
numRecords: Int = 1,
|
||||
|
|
|
@ -57,6 +57,11 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with
|
|||
createScramCredentials(zkConnect, JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword)
|
||||
}
|
||||
|
||||
override def createPrivilegedAdminClient() = {
|
||||
createAdminClient(brokerList, securityProtocol, trustStoreFile, clientSaslProperties,
|
||||
kafkaClientSaslMechanism, JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword)
|
||||
}
|
||||
|
||||
@Before
|
||||
override def setUp(): Unit = {
|
||||
startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), Both,
|
||||
|
@ -202,7 +207,7 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with
|
|||
}
|
||||
|
||||
private def createClientCredential(): Unit = {
|
||||
createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
|
||||
createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
|
||||
}
|
||||
|
||||
private def sendOneRecord(producer: KafkaProducer[Array[Byte], Array[Byte]], maxWaitMs: Long = 15000): Unit = {
|
||||
|
|
|
@ -38,11 +38,13 @@ class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTes
|
|||
createScramCredentials(zkConnect, kafkaPrincipal.getName, kafkaPassword)
|
||||
}
|
||||
|
||||
override def createPrivilegedAdminClient() = createScramAdminClient(kafkaClientSaslMechanism, kafkaPrincipal.getName, kafkaPassword)
|
||||
|
||||
@Before
|
||||
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)
|
||||
createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
|
||||
createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KafkaScramUser2, JaasTestUtils.KafkaScramPassword2)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,7 +18,9 @@
|
|||
package kafka.api
|
||||
|
||||
import java.io.File
|
||||
import java.util
|
||||
import java.util.Properties
|
||||
|
||||
import javax.security.auth.login.Configuration
|
||||
|
||||
import scala.collection.Seq
|
||||
|
@ -27,14 +29,18 @@ import kafka.server.{ConfigType, KafkaConfig}
|
|||
import kafka.utils.JaasTestUtils.{JaasSection, Krb5LoginModule, ZkDigestModule}
|
||||
import kafka.utils.{JaasTestUtils, TestUtils}
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, ScramCredentialInfo, UserScramCredentialAlteration, UserScramCredentialUpsertion, ScramMechanism => PublicScramMechanism}
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
|
||||
import org.apache.kafka.common.security.JaasUtils
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.security.authenticator.LoginManager
|
||||
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism}
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
/*
|
||||
* Implements an enumeration for the modes enabled here:
|
||||
* zk only, kafka only, both, custom KafkaServer.
|
||||
|
@ -148,6 +154,43 @@ trait SaslSetup {
|
|||
JaasTestUtils.clientLoginModule(clientSaslMechanism, clientKeytabFile)
|
||||
}
|
||||
|
||||
def jaasScramClientLoginModule(clientSaslScramMechanism: String, scramUser: String, scramPassword: String): String = {
|
||||
JaasTestUtils.scramClientLoginModule(clientSaslScramMechanism, scramUser, scramPassword)
|
||||
}
|
||||
|
||||
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 createAdminClient(brokerList: String, securityProtocol: SecurityProtocol, trustStoreFile: Option[File],
|
||||
clientSaslProperties: Option[Properties], scramMechanism: String, user: String, password: String) : Admin = {
|
||||
val config = new util.HashMap[String, Object]
|
||||
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
|
||||
val securityProps: util.Map[Object, Object] =
|
||||
TestUtils.adminClientSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties)
|
||||
securityProps.forEach { (key, value) => config.put(key.asInstanceOf[String], value) }
|
||||
config.put(SaslConfigs.SASL_JAAS_CONFIG, jaasScramClientLoginModule(scramMechanism, user, password))
|
||||
Admin.create(config)
|
||||
}
|
||||
|
||||
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()
|
||||
}
|
||||
}
|
||||
|
||||
def createScramCredentials(adminClient: Admin, userName: String, password: String): Unit = {
|
||||
val results = adminClient.alterUserScramCredentials(PublicScramMechanism.values().filter(_ != PublicScramMechanism.UNKNOWN).map(mechanism =>
|
||||
new UserScramCredentialUpsertion(userName, new ScramCredentialInfo(mechanism, 4096), password)
|
||||
.asInstanceOf[UserScramCredentialAlteration]).toList.asJava)
|
||||
results.all.get
|
||||
}
|
||||
|
||||
def createScramCredentials(zkConnect: String, userName: String, password: String): Unit = {
|
||||
val zkClientConfig = new ZKClientConfig()
|
||||
val zkClient = KafkaZkClient(
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.kafka.common.network.{ListenerName, Mode}
|
|||
import org.apache.kafka.common.network.CertStores.{KEYSTORE_PROPS, TRUSTSTORE_PROPS}
|
||||
import org.apache.kafka.common.record.TimestampType
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.security.scram.ScramCredential
|
||||
import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer}
|
||||
import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
|
||||
import org.junit.Assert._
|
||||
|
@ -1047,9 +1048,18 @@ 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()
|
||||
// make sure each server's credential cache has all the created credentials
|
||||
// (check after initializing Kerberos to minimize delays)
|
||||
List(JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramAdmin).foreach { scramUser =>
|
||||
servers.foreach { server =>
|
||||
ScramMechanism.values().filter(_ != ScramMechanism.UNKNOWN).foreach(mechanism =>
|
||||
TestUtils.waitUntilTrue(() => server.credentialProvider.credentialCache.cache(
|
||||
mechanism.mechanismName(), classOf[ScramCredential]).get(scramUser) != null,
|
||||
s"$mechanism credentials not created for $scramUser"))
|
||||
}}
|
||||
|
||||
//verifyAddListener("SASL_SSL", SecurityProtocol.SASL_SSL, Seq("SCRAM-SHA-512", "SCRAM-SHA-256", "PLAIN"))
|
||||
verifyAddListener("SASL_PLAINTEXT", SecurityProtocol.SASL_PLAINTEXT, Seq("GSSAPI"))
|
||||
|
|
|
@ -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
|
||||
val node = new Node(1, "localhost", 9092)
|
||||
val mockAdminClient = new MockAdminClient(util.Collections.singletonList(node), node)
|
||||
|
||||
|
@ -501,9 +503,126 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
|
|||
}
|
||||
|
||||
verifyCommand("users", "--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10")
|
||||
verifyCommand("users", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret],some_config=10")
|
||||
verifyCommand("clients", "--add-config", "some_config=10")
|
||||
verifyCommand("users", "--delete-config", "consumer_byte_rate=20000,some_config=10")
|
||||
verifyCommand("clients", "--delete-config", "some_config=10")
|
||||
verifyCommand("users", "--delete-config", "consumer_byte_rate,some_config")
|
||||
verifyCommand("users", "--delete-config", "SCRAM-SHA-256,some_config")
|
||||
verifyCommand("clients", "--delete-config", "some_config")
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldNotAlterScramClientConfigUsingBootstrapServer(): Unit = {
|
||||
// when using --bootstrap-server, it should be illegal to alter SCRAM credentials for client entities
|
||||
val node = new Node(1, "localhost", 9092)
|
||||
val mockAdminClient = new MockAdminClient(util.Collections.singletonList(node), node)
|
||||
|
||||
def verifyCommand(entityType: String, alterOpts: String*): Unit = {
|
||||
val opts = new ConfigCommandOptions(Array("--bootstrap-server", "localhost:9092",
|
||||
"--entity-type", entityType, "--entity-name", "admin",
|
||||
"--alter") ++ alterOpts)
|
||||
val e = intercept[IllegalArgumentException] {
|
||||
ConfigCommand.alterConfig(mockAdminClient, opts)
|
||||
}
|
||||
assertTrue(s"Unexpected exception: $e", e.getMessage.contains("SCRAM-SHA-256"))
|
||||
}
|
||||
|
||||
verifyCommand("clients", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]")
|
||||
verifyCommand("clients", "--delete-config", "SCRAM-SHA-256")
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldNotCreateUserScramCredentialConfigWithUnderMinimumIterationsUsingBootstrapServer(): Unit = {
|
||||
// when using --bootstrap-server, it should be illegal to create a SCRAM credential for a user
|
||||
// with an iterations value less than the minimum
|
||||
val node = new Node(1, "localhost", 9092)
|
||||
val mockAdminClient = new MockAdminClient(util.Collections.singletonList(node), node)
|
||||
|
||||
def verifyCommand(entityType: String, alterOpts: String*): Unit = {
|
||||
val opts = new ConfigCommandOptions(Array("--bootstrap-server", "localhost:9092",
|
||||
"--entity-type", entityType, "--entity-name", "admin",
|
||||
"--alter") ++ alterOpts)
|
||||
val e = intercept[IllegalArgumentException] {
|
||||
ConfigCommand.alterConfig(mockAdminClient, opts)
|
||||
}
|
||||
assertTrue(s"Unexpected exception: $e", e.getMessage.contains("SCRAM-SHA-256"))
|
||||
}
|
||||
|
||||
verifyCommand("users", "--add-config", "SCRAM-SHA-256=[iterations=100,password=foo-secret]")
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldNotAlterUserScramCredentialAndClientQuotaConfigsSimultaneouslyUsingBootstrapServer(): Unit = {
|
||||
// when using --bootstrap-server, it should be illegal to alter both SCRAM credentials and quotas for user entities
|
||||
val node = new Node(1, "localhost", 9092)
|
||||
val mockAdminClient = new MockAdminClient(util.Collections.singletonList(node), node)
|
||||
|
||||
def verifyCommand(alterOpts: String*): Unit = {
|
||||
val opts = new ConfigCommandOptions(Array("--bootstrap-server", "localhost:9092", "--alter") ++ alterOpts)
|
||||
val e = intercept[IllegalArgumentException] {
|
||||
ConfigCommand.alterConfig(mockAdminClient, opts)
|
||||
}
|
||||
assertTrue(s"Unexpected exception: $e", e.getMessage.contains("SCRAM-SHA-256"))
|
||||
}
|
||||
|
||||
verifyCommand("--entity-type", "users", "--entity-name", "admin", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]",
|
||||
"--entity-type", "users", "--entity-name", "admin", "--delete-config", "consumer_byte_rate")
|
||||
verifyCommand("--entity-type", "users", "--entity-name", "admin", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]",
|
||||
"--entity-type", "users", "--entity-name", "admin1", "--delete-config", "consumer_byte_rate")
|
||||
verifyCommand("--entity-type", "users", "--entity-name", "admin", "--delete-config", "SCRAM-SHA-256",
|
||||
"--entity-type", "users", "--entity-name", "admin", "--add-config", "consumer_byte_rate=20000")
|
||||
verifyCommand("--entity-type", "users", "--entity-name", "admin", "--delete-config", "SCRAM-SHA-256",
|
||||
"--entity-type", "users", "--entity-name", "admin1", "--add-config", "consumer_byte_rate=20000")
|
||||
|
||||
verifyCommand("--entity-type", "clients", "--entity-name", "admin", "--delete-config", "consumer_byte_rate",
|
||||
"--entity-type", "users", "--entity-name", "admin", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]")
|
||||
verifyCommand( "--entity-type", "clients", "--entity-name", "admin1", "--delete-config", "consumer_byte_rate",
|
||||
"--entity-type", "users", "--entity-name", "admin", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]")
|
||||
verifyCommand("--entity-type", "clients", "--entity-name", "admin", "--add-config", "consumer_byte_rate=20000",
|
||||
"--entity-type", "users", "--entity-name", "admin", "--delete-config", "SCRAM-SHA-256")
|
||||
verifyCommand("--entity-type", "users", "--entity-name", "admin1", "--add-config", "consumer_byte_rate=20000",
|
||||
"--entity-type", "users", "--entity-name", "admin", "--delete-config", "SCRAM-SHA-256")
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldNotDescribeUserScramCredentialsWithEntityDefaultUsingBootstrapServer(): Unit = {
|
||||
// User SCRAM credentials should not be described when specifying
|
||||
// --describe --entity-type users --entity-default (or --user-defaults) with --bootstrap-server
|
||||
val describeFuture = new KafkaFutureImpl[util.Map[ClientQuotaEntity, util.Map[String, java.lang.Double]]]
|
||||
describeFuture.complete(Map((new ClientQuotaEntity(Map("" -> "").asJava) -> Map(("request_percentage" -> Double.box(50.0))).asJava)).asJava)
|
||||
val describeClientQuotasResult: DescribeClientQuotasResult = EasyMock.createNiceMock(classOf[DescribeClientQuotasResult])
|
||||
EasyMock.expect(describeClientQuotasResult.entities()).andReturn(describeFuture).times(2)
|
||||
EasyMock.replay(describeClientQuotasResult)
|
||||
|
||||
val node = new Node(1, "localhost", 9092)
|
||||
val mockAdminClient = new MockAdminClient(util.Collections.singletonList(node), node) {
|
||||
override def describeClientQuotas(filter: ClientQuotaFilter, options: DescribeClientQuotasOptions): DescribeClientQuotasResult = {
|
||||
describeClientQuotasResult
|
||||
}
|
||||
override def describeUserScramCredentials(users: util.List[String], options: DescribeUserScramCredentialsOptions): DescribeUserScramCredentialsResult = {
|
||||
throw new IllegalStateException("Incorrectly described SCRAM credentials when specifying --entity-default with --bootstrap-server")
|
||||
}
|
||||
}
|
||||
|
||||
def verifyCommand(expectedMessage: String, alterOrDescribeOpt: String, requestOpts: String*): Unit = {
|
||||
val opts = new ConfigCommandOptions(Array("--bootstrap-server", "localhost:9092",
|
||||
alterOrDescribeOpt) ++ requestOpts)
|
||||
if (alterOrDescribeOpt.equals("--describe"))
|
||||
ConfigCommand.describeConfig(mockAdminClient, opts) // fails if describeUserScramCredentials() is invoked
|
||||
else {
|
||||
val e = intercept[IllegalArgumentException] {
|
||||
ConfigCommand.alterConfig(mockAdminClient, opts)
|
||||
}
|
||||
assertTrue(s"Unexpected exception: $e", e.getMessage.contains(expectedMessage))
|
||||
}
|
||||
}
|
||||
|
||||
val expectedMsg = "The use of --entity-default or --user-defaults is not allowed with User SCRAM Credentials using --bootstrap-server."
|
||||
verifyCommand(expectedMsg, "--alter", "--entity-type", "users", "--entity-default", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]")
|
||||
verifyCommand(expectedMsg, "--alter", "--entity-type", "users", "--entity-default", "--delete-config", "SCRAM-SHA-256")
|
||||
verifyCommand(expectedMsg, "--describe", "--entity-type", "users", "--entity-default")
|
||||
verifyCommand(expectedMsg, "--alter", "--user-defaults", "--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]")
|
||||
verifyCommand(expectedMsg, "--alter", "--user-defaults", "--delete-config", "SCRAM-SHA-256")
|
||||
verifyCommand(expectedMsg, "--describe", "--user-defaults")
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -0,0 +1,137 @@
|
|||
/**
|
||||
* 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
|
||||
}
|
||||
val commandArgs = Array("--bootstrap-server", brokerList) ++ args
|
||||
try {
|
||||
Console.withOut(printStream) {
|
||||
ConfigCommand.main(commandArgs)
|
||||
}
|
||||
ConfigCommandResult(byteArrayOutputStream.toString(utf8))
|
||||
} catch {
|
||||
case e: Exception => {
|
||||
debug(s"Exception running ConfigCommand ${commandArgs.mkString(" ")}", e)
|
||||
ConfigCommandResult("", exitStatus)
|
||||
}
|
||||
} finally {
|
||||
printStream.close
|
||||
Exit.resetExitProcedure()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testUserScramCredentialsRequests(): Unit = {
|
||||
val user1 = "user1"
|
||||
// create and describe a credential
|
||||
var result = runConfigCommandViaBroker(Array("--user", user1, "--alter", "--add-config", "SCRAM-SHA-256=[iterations=4096,password=foo-secret]"))
|
||||
val alterConfigsUser1Out = s"Completed updating config for user $user1.\n"
|
||||
assertEquals(alterConfigsUser1Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--user", user1, "--describe"))
|
||||
val scramCredentialConfigsUser1Out = s"SCRAM credential configs for user-principal '$user1' are SCRAM-SHA-256=iterations=4096\n"
|
||||
assertEquals(scramCredentialConfigsUser1Out, result.stdout)
|
||||
// create a user quota and describe the user again
|
||||
result = runConfigCommandViaBroker(Array("--user", user1, "--alter", "--add-config", "consumer_byte_rate=20000"))
|
||||
assertEquals(alterConfigsUser1Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--user", user1, "--describe"))
|
||||
val quotaConfigsUser1Out = s"Quota configs for user-principal '$user1' are consumer_byte_rate=20000.0\n"
|
||||
assertEquals(s"$quotaConfigsUser1Out$scramCredentialConfigsUser1Out", result.stdout)
|
||||
|
||||
// now do the same thing for user2
|
||||
val user2 = "user2"
|
||||
// create and describe a credential
|
||||
result = runConfigCommandViaBroker(Array("--user", user2, "--alter", "--add-config", "SCRAM-SHA-256=[iterations=4096,password=foo-secret]"))
|
||||
val alterConfigsUser2Out = s"Completed updating config for user $user2.\n"
|
||||
assertEquals(alterConfigsUser2Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--user", user2, "--describe"))
|
||||
val scramCredentialConfigsUser2Out = s"SCRAM credential configs for user-principal '$user2' are SCRAM-SHA-256=iterations=4096\n"
|
||||
assertEquals(scramCredentialConfigsUser2Out, result.stdout)
|
||||
// create a user quota and describe the user again
|
||||
result = runConfigCommandViaBroker(Array("--user", user2, "--alter", "--add-config", "consumer_byte_rate=20000"))
|
||||
assertEquals(alterConfigsUser2Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--user", user2, "--describe"))
|
||||
val quotaConfigsUser2Out = s"Quota configs for user-principal '$user2' are consumer_byte_rate=20000.0\n"
|
||||
assertEquals(s"$quotaConfigsUser2Out$scramCredentialConfigsUser2Out", result.stdout)
|
||||
|
||||
// describe both
|
||||
result = runConfigCommandViaBroker(Array("--entity-type", "users", "--describe"))
|
||||
// we don't know the order that quota or scram users come out, so we have 2 possibilities for each, 4 total
|
||||
val quotaPossibilityAOut = s"$quotaConfigsUser1Out$quotaConfigsUser2Out"
|
||||
val quotaPossibilityBOut = s"$quotaConfigsUser2Out$quotaConfigsUser1Out"
|
||||
val scramPossibilityAOut = s"$scramCredentialConfigsUser1Out$scramCredentialConfigsUser2Out"
|
||||
val scramPossibilityBOut = s"$scramCredentialConfigsUser2Out$scramCredentialConfigsUser1Out"
|
||||
assertTrue(result.stdout.equals(s"$quotaPossibilityAOut$scramPossibilityAOut")
|
||||
|| result.stdout.equals(s"$quotaPossibilityAOut$scramPossibilityBOut")
|
||||
|| result.stdout.equals(s"$quotaPossibilityBOut$scramPossibilityAOut")
|
||||
|| result.stdout.equals(s"$quotaPossibilityBOut$scramPossibilityBOut"))
|
||||
|
||||
// now delete configs, in opposite order, for user1 and user2, and describe
|
||||
result = runConfigCommandViaBroker(Array("--user", user1, "--alter", "--delete-config", "consumer_byte_rate"))
|
||||
assertEquals(alterConfigsUser1Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--user", user2, "--alter", "--delete-config", "SCRAM-SHA-256"))
|
||||
assertEquals(alterConfigsUser2Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--entity-type", "users", "--describe"))
|
||||
assertEquals(s"$quotaConfigsUser2Out$scramCredentialConfigsUser1Out", result.stdout)
|
||||
|
||||
// now delete the rest of the configs, for user1 and user2, and describe
|
||||
result = runConfigCommandViaBroker(Array("--user", user1, "--alter", "--delete-config", "SCRAM-SHA-256"))
|
||||
assertEquals(alterConfigsUser1Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--user", user2, "--alter", "--delete-config", "consumer_byte_rate"))
|
||||
assertEquals(alterConfigsUser2Out, result.stdout)
|
||||
result = runConfigCommandViaBroker(Array("--entity-type", "users", "--describe"))
|
||||
assertEquals("", result.stdout)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAlterWithEmptyPassword(): Unit = {
|
||||
val user1 = "user1"
|
||||
val result = runConfigCommandViaBroker(Array("--user", user1, "--alter", "--add-config", "SCRAM-SHA-256=[iterations=4096,password=]"))
|
||||
assertTrue("Expected System.exit() to be called with an empty password", result.exitStatus.isDefined)
|
||||
assertEquals("Expected empty password to cause failure with exit status=1", 1, result.exitStatus.get)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDescribeUnknownUser(): Unit = {
|
||||
val unknownUser = "unknownUser"
|
||||
val result = runConfigCommandViaBroker(Array("--user", unknownUser, "--describe"))
|
||||
assertTrue("Expected System.exit() to not be called with an unknown user", result.exitStatus.isEmpty)
|
||||
assertEquals("", result.stdout)
|
||||
}
|
||||
}
|
|
@ -32,6 +32,7 @@ import java.util.Properties
|
|||
|
||||
import org.apache.kafka.common.KafkaException
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.security.scram.ScramCredential
|
||||
import org.apache.kafka.common.utils.Time
|
||||
|
||||
/**
|
||||
|
@ -159,4 +160,12 @@ abstract class KafkaServerTestHarness extends ZooKeeperTestHarness {
|
|||
alive(i) = true
|
||||
}
|
||||
}
|
||||
|
||||
def waitForUserScramCredentialToAppearOnAllBrokers(clientPrincipal: String, mechanismName: String): Unit = {
|
||||
servers.foreach { server =>
|
||||
val cache = server.credentialProvider.credentialCache.cache(mechanismName, classOf[ScramCredential])
|
||||
TestUtils.waitUntilTrue(() => cache.get(clientPrincipal) != null, s"SCRAM credentials not created for $clientPrincipal")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,433 @@
|
|||
/*
|
||||
* 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.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult
|
||||
import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, DescribeUserScramCredentialsRequestData}
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse, DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
|
||||
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.rules.TestName
|
||||
import org.junit.{Rule, 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,
|
||||
if (testName.getMethodName.endsWith("NotAuthorized")) {
|
||||
classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName
|
||||
} else {
|
||||
classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName
|
||||
})
|
||||
}
|
||||
|
||||
private val _testName = new TestName
|
||||
@Rule def testName = _testName
|
||||
|
||||
private val saltedPasswordBytes = "saltedPassword".getBytes(StandardCharsets.UTF_8)
|
||||
private val saltBytes = "salt".getBytes(StandardCharsets.UTF_8)
|
||||
private val user1 = "user1"
|
||||
private val user2 = "user2"
|
||||
private val unknownUser = "unknownUser"
|
||||
|
||||
@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 = {
|
||||
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 = {
|
||||
|
||||
val request = new AlterUserScramCredentialsRequest.Builder(
|
||||
new AlterUserScramCredentialsRequestData()
|
||||
.setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
|
||||
.setUpsertions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)))).build()
|
||||
val response = sendAlterUserScramCredentialsRequest(request)
|
||||
|
||||
val results = response.data.results
|
||||
assertEquals(2, results.size)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.CLUSTER_AUTHORIZATION_FAILED, "when not authorized")
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAlterSameThingTwice(): Unit = {
|
||||
val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||
val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||
val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||
.setIterations(4096).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
val upsertion2 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||
.setIterations(4096).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
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)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.DUPLICATE_RESOURCE, "when altering the same credential twice in a single request")
|
||||
})
|
||||
}
|
||||
|
||||
@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(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
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)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.UNACCEPTABLE_CREDENTIAL, "when altering an empty user")
|
||||
assertEquals("Username must not be empty", results.get(0).errorMessage)
|
||||
})
|
||||
}
|
||||
|
||||
@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(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
val upsertionValid3 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user3).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||
.setIterations(8192).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
val user4 = "user4"
|
||||
val upsertionUnknown4 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user4).setMechanism(10.toByte)
|
||||
.setIterations(8192).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
val user5 = "user5"
|
||||
val upsertionUnknown5 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user5).setMechanism(ScramMechanism.UNKNOWN.`type`)
|
||||
.setIterations(8192).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
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)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.UNSUPPORTED_SASL_MECHANISM, "when altering the credentials with unknown SCRAM mechanisms")
|
||||
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(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
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)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.UNACCEPTABLE_CREDENTIAL, "when altering the credentials with too few iterations")
|
||||
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(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||
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)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.UNACCEPTABLE_CREDENTIAL, "when altering the credentials with too many iterations")
|
||||
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(user1).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)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.RESOURCE_NOT_FOUND, "when deleting a non-existing credential")
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAlterNotController(): Unit = {
|
||||
val request = new AlterUserScramCredentialsRequest.Builder(
|
||||
new AlterUserScramCredentialsRequestData()
|
||||
.setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
|
||||
.setUpsertions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)))).build()
|
||||
val response = sendAlterUserScramCredentialsRequest(request, notControllerSocketServer)
|
||||
|
||||
val results = response.data.results
|
||||
assertEquals(2, results.size)
|
||||
checkAllErrorsAlteringCredentials(results, Errors.NOT_CONTROLLER, "when routed incorrectly to a non-Controller broker")
|
||||
}
|
||||
|
||||
@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(saltBytes)
|
||||
.setSaltedPassword(saltedPasswordBytes),
|
||||
new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion()
|
||||
.setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)
|
||||
.setIterations(8192)
|
||||
.setSalt(saltBytes)
|
||||
.setSaltedPassword(saltedPasswordBytes),
|
||||
new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion()
|
||||
.setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_512.`type`)
|
||||
.setIterations(8192)
|
||||
.setSalt(saltBytes)
|
||||
.setSaltedPassword(saltedPasswordBytes),
|
||||
))).build()
|
||||
val results1 = sendAlterUserScramCredentialsRequest(request1).data.results
|
||||
assertEquals(2, results1.size)
|
||||
checkNoErrorsAlteringCredentials(results1)
|
||||
checkUserAppearsInAlterResults(results1, user1)
|
||||
checkUserAppearsInAlterResults(results1, user2)
|
||||
|
||||
// now describe them all
|
||||
val results2 = describeAllWithNoTopLevelErrorConfirmed().data.results
|
||||
assertEquals(2, results2.size)
|
||||
checkUserHasTwoCredentials(results2, user1)
|
||||
checkForSingleSha512Iterations8192Credential(results2, user2)
|
||||
|
||||
// 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)
|
||||
checkNoTopLevelErrorDescribingCredentials(response3)
|
||||
val results3 = response3.data.results
|
||||
assertEquals(1, results3.size)
|
||||
checkUserHasTwoCredentials(results3, user1)
|
||||
|
||||
// now test per-user errors by describing user1 and an unknown
|
||||
val requestUnknown = new DescribeUserScramCredentialsRequest.Builder(
|
||||
new DescribeUserScramCredentialsRequestData().setUsers(util.Arrays.asList(
|
||||
new DescribeUserScramCredentialsRequestData.UserName().setName(user1),
|
||||
new DescribeUserScramCredentialsRequestData.UserName().setName(unknownUser)))).build()
|
||||
val responseUnknown = sendDescribeUserScramCredentialsRequest(requestUnknown)
|
||||
checkNoTopLevelErrorDescribingCredentials(responseUnknown)
|
||||
val resultsUnknown = responseUnknown.data.results
|
||||
assertEquals(2, resultsUnknown.size)
|
||||
checkUserHasTwoCredentials(resultsUnknown, user1)
|
||||
checkDescribeForError(resultsUnknown, unknownUser, Errors.RESOURCE_NOT_FOUND)
|
||||
|
||||
// now test per-user errors again by describing user1 along with user2 twice
|
||||
val requestDuplicateUser = new DescribeUserScramCredentialsRequest.Builder(
|
||||
new DescribeUserScramCredentialsRequestData().setUsers(util.Arrays.asList(
|
||||
new DescribeUserScramCredentialsRequestData.UserName().setName(user1),
|
||||
new DescribeUserScramCredentialsRequestData.UserName().setName(user2),
|
||||
new DescribeUserScramCredentialsRequestData.UserName().setName(user2)))).build()
|
||||
val responseDuplicateUser = sendDescribeUserScramCredentialsRequest(requestDuplicateUser)
|
||||
checkNoTopLevelErrorDescribingCredentials(responseDuplicateUser)
|
||||
val resultsDuplicateUser = responseDuplicateUser.data.results
|
||||
assertEquals(2, resultsDuplicateUser.size)
|
||||
checkUserHasTwoCredentials(resultsDuplicateUser, user1)
|
||||
checkDescribeForError(resultsDuplicateUser, user2, Errors.DUPLICATE_RESOURCE)
|
||||
|
||||
// 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)
|
||||
checkNoErrorsAlteringCredentials(results4)
|
||||
checkUserAppearsInAlterResults(results4, user1)
|
||||
checkUserAppearsInAlterResults(results4, user2)
|
||||
|
||||
// now describe them all, which should just yield 1 credential
|
||||
val results5 = describeAllWithNoTopLevelErrorConfirmed().data.results
|
||||
assertEquals(1, results5.size)
|
||||
checkForSingleSha512Iterations8192Credential(results5, user1)
|
||||
|
||||
// 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 results6 = sendAlterUserScramCredentialsRequest(request6).data.results
|
||||
assertEquals(1, results6.size)
|
||||
checkNoErrorsAlteringCredentials(results6)
|
||||
checkUserAppearsInAlterResults(results6, user1)
|
||||
|
||||
// now describe them all, which should yield 0 credentials
|
||||
val results7 = describeAllWithNoTopLevelErrorConfirmed().data.results
|
||||
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)
|
||||
}
|
||||
|
||||
private def checkAllErrorsAlteringCredentials(resultsToCheck: util.List[AlterUserScramCredentialsResult], expectedError: Errors, contextMsg: String) = {
|
||||
assertEquals(s"Expected all '${expectedError.name}' errors when altering credentials $contextMsg",
|
||||
0, resultsToCheck.asScala.filterNot(_.errorCode == expectedError.code).size)
|
||||
}
|
||||
|
||||
private def checkNoErrorsAlteringCredentials(resultsToCheck: util.List[AlterUserScramCredentialsResult]) = {
|
||||
assertEquals("Expected no error when altering credentials",
|
||||
0, resultsToCheck.asScala.filterNot(_.errorCode == Errors.NONE.code).size)
|
||||
}
|
||||
|
||||
private def checkUserAppearsInAlterResults(resultsToCheck: util.List[AlterUserScramCredentialsResult], user: String) = {
|
||||
assertTrue(s"Expected result to contain '$user'", resultsToCheck.asScala.exists(_.user == user))
|
||||
}
|
||||
|
||||
private def describeAllWithNoTopLevelErrorConfirmed() = {
|
||||
val response = sendDescribeUserScramCredentialsRequest(
|
||||
new DescribeUserScramCredentialsRequest.Builder(new DescribeUserScramCredentialsRequestData()).build())
|
||||
checkNoTopLevelErrorDescribingCredentials(response)
|
||||
response
|
||||
}
|
||||
|
||||
private def checkNoTopLevelErrorDescribingCredentials(responseToCheck: DescribeUserScramCredentialsResponse) = {
|
||||
assertEquals("Expected no top-level error when describing the credentials", Errors.NONE.code, responseToCheck.data.errorCode)
|
||||
}
|
||||
|
||||
private def checkUserHasTwoCredentials(resultsToCheck: util.List[DescribeUserScramCredentialsResult], user: String) = {
|
||||
assertTrue(s"Expected result to contain '$user' with 2 credentials: $resultsToCheck",
|
||||
resultsToCheck.asScala.exists(result => result.user == user && result.credentialInfos.size == 2 && result.errorCode == Errors.NONE.code))
|
||||
assertTrue(s"Expected result to contain '$user' with SCRAM_SHA_256/4096 and SCRAM_SHA_512/8192 credentials: $resultsToCheck",
|
||||
resultsToCheck.asScala.exists(result => result.user == user && result.credentialInfos.asScala.exists(info =>
|
||||
info.mechanism == ScramMechanism.SCRAM_SHA_256.`type` && info.iterations == 4096)
|
||||
&& result.credentialInfos.asScala.exists(info =>
|
||||
info.mechanism == ScramMechanism.SCRAM_SHA_512.`type` && info.iterations == 8192)))
|
||||
}
|
||||
|
||||
private def checkForSingleSha512Iterations8192Credential(resultsToCheck: util.List[DescribeUserScramCredentialsResult], user: String) = {
|
||||
assertTrue(s"Expected result to contain '$user' with 1 credential: $resultsToCheck",
|
||||
resultsToCheck.asScala.exists(result => result.user == user && result.credentialInfos.size == 1 && result.errorCode == Errors.NONE.code))
|
||||
assertTrue(s"Expected result to contain '$user' with SCRAM_SHA_512/8192 credential: $resultsToCheck",
|
||||
resultsToCheck.asScala.exists(result => result.user == user && result.credentialInfos.asScala.exists(info =>
|
||||
info.mechanism == ScramMechanism.SCRAM_SHA_512.`type` && info.iterations == 8192)))
|
||||
}
|
||||
|
||||
private def checkDescribeForError(resultsToCheck: util.List[DescribeUserScramCredentialsResult], user: String, expectedError: Errors) = {
|
||||
assertTrue(s"Expected result to contain '$user' with a ${expectedError.name} error: $resultsToCheck",
|
||||
resultsToCheck.asScala.exists(result => result.user == user && result.credentialInfos.size == 0 && result.errorCode == expectedError.code))
|
||||
}
|
||||
}
|
||||
|
||||
object AlterCredentialsTest {
|
||||
val UnauthorizedPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "Unauthorized")
|
||||
val AuthorizedPrincipal = KafkaPrincipal.ANONYMOUS
|
||||
|
||||
class TestAuthorizer extends AclAuthorizer {
|
||||
override def authorize(requestContext: AuthorizableRequestContext, actions: util.List[Action]): util.List[AuthorizationResult] = {
|
||||
actions.asScala.map { _ =>
|
||||
if (requestContext.requestType == ApiKeys.ALTER_USER_SCRAM_CREDENTIALS.id && requestContext.principal == UnauthorizedPrincipal)
|
||||
AuthorizationResult.DENIED
|
||||
else
|
||||
AuthorizationResult.ALLOWED
|
||||
}.asJava
|
||||
}
|
||||
}
|
||||
|
||||
class TestPrincipalBuilderReturningAuthorized extends KafkaPrincipalBuilder {
|
||||
override def build(context: AuthenticationContext): KafkaPrincipal = {
|
||||
AuthorizedPrincipal
|
||||
}
|
||||
}
|
||||
|
||||
class TestPrincipalBuilderReturningUnauthorized extends KafkaPrincipalBuilder {
|
||||
override def build(context: AuthenticationContext): KafkaPrincipal = {
|
||||
UnauthorizedPrincipal
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,16 +17,16 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import org.apache.kafka.clients.admin.{ScramCredentialInfo, ScramMechanism, UserScramCredentialUpsertion}
|
||||
import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException}
|
||||
import org.apache.kafka.common.internals.KafkaFutureImpl
|
||||
import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent}
|
||||
import org.apache.kafka.common.requests.{AlterClientQuotasRequest, AlterClientQuotasResponse, DescribeClientQuotasRequest, DescribeClientQuotasResponse}
|
||||
import org.junit.Assert._
|
||||
import org.junit.Test
|
||||
import java.util
|
||||
import java.util.concurrent.{ExecutionException, TimeUnit}
|
||||
|
||||
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
class ClientQuotasRequestTest extends BaseRequestTest {
|
||||
|
@ -166,14 +166,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(
|
||||
new UserScramCredentialUpsertion(userName, new ScramCredentialInfo(ScramMechanism.SCRAM_SHA_256, 4096), "password")))
|
||||
results.all.get
|
||||
|
||||
val entity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> userName).asJava)
|
||||
|
||||
|
|
|
@ -0,0 +1,149 @@
|
|||
/*
|
||||
* 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.message.{DescribeUserScramCredentialsRequestData, DescribeUserScramCredentialsResponseData}
|
||||
import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName
|
||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse}
|
||||
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.rules.TestName
|
||||
import org.junit.{Rule, 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,
|
||||
if (testName.getMethodName.endsWith("NotAuthorized")) {
|
||||
classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName
|
||||
} else {
|
||||
classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName
|
||||
})
|
||||
}
|
||||
|
||||
private val _testName = new TestName
|
||||
@Rule def testName = _testName
|
||||
|
||||
@Test
|
||||
def testDescribeNothing(): Unit = {
|
||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||
new DescribeUserScramCredentialsRequestData()).build()
|
||||
val response = sendDescribeUserScramCredentialsRequest(request)
|
||||
|
||||
val error = response.data.errorCode
|
||||
assertEquals("Expected no error when describing everything and there are no credentials",
|
||||
Errors.NONE.code, error)
|
||||
assertEquals("Expected no credentials when describing everything and there are no credentials",
|
||||
0, response.data.results.size)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDescribeNotController(): Unit = {
|
||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||
new DescribeUserScramCredentialsRequestData()).build()
|
||||
val response = sendDescribeUserScramCredentialsRequest(request, notControllerSocketServer)
|
||||
|
||||
val error = response.data.errorCode
|
||||
assertEquals("Did not expect controller error when routed to non-controller", Errors.NONE.code, error)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDescribeNotAuthorized(): Unit = {
|
||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||
new DescribeUserScramCredentialsRequestData()).build()
|
||||
val response = sendDescribeUserScramCredentialsRequest(request)
|
||||
|
||||
val error = response.data.errorCode
|
||||
assertEquals("Expected not authorized error", Errors.CLUSTER_AUTHORIZATION_FAILED.code, error)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDescribeSameUserTwice(): Unit = {
|
||||
val user = "user1"
|
||||
val userName = new UserName().setName(user)
|
||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||
new DescribeUserScramCredentialsRequestData().setUsers(List(userName, userName).asJava)).build()
|
||||
val response = sendDescribeUserScramCredentialsRequest(request)
|
||||
|
||||
assertEquals("Expected no top-level error", Errors.NONE.code, response.data.errorCode)
|
||||
assertEquals(1, response.data.results.size)
|
||||
val result: DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult = response.data.results.get(0)
|
||||
assertEquals(s"Expected duplicate resource error for $user", Errors.DUPLICATE_RESOURCE.code, result.errorCode)
|
||||
assertEquals(s"Cannot describe SCRAM credentials for the same user twice in a single request: $user", result.errorMessage)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testUnknownUser(): Unit = {
|
||||
val unknownUser = "unknownUser"
|
||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||
new DescribeUserScramCredentialsRequestData().setUsers(List(new UserName().setName(unknownUser)).asJava)).build()
|
||||
val response = sendDescribeUserScramCredentialsRequest(request)
|
||||
|
||||
assertEquals("Expected no top-level error", Errors.NONE.code, response.data.errorCode)
|
||||
assertEquals(1, response.data.results.size)
|
||||
val result: DescribeUserScramCredentialsResponseData.DescribeUserScramCredentialsResult = response.data.results.get(0)
|
||||
assertEquals(s"Expected duplicate resource error for $unknownUser", Errors.RESOURCE_NOT_FOUND.code, result.errorCode)
|
||||
assertEquals(s"Attempt to describe a user credential that does not exist: $unknownUser", result.errorMessage)
|
||||
}
|
||||
|
||||
private def sendDescribeUserScramCredentialsRequest(request: DescribeUserScramCredentialsRequest, socketServer: SocketServer = controllerSocketServer): DescribeUserScramCredentialsResponse = {
|
||||
connectAndReceive[DescribeUserScramCredentialsResponse](request, destination = socketServer)
|
||||
}
|
||||
}
|
||||
|
||||
object DescribeCredentialsTest {
|
||||
val UnauthorizedPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "Unauthorized")
|
||||
val AuthorizedPrincipal = KafkaPrincipal.ANONYMOUS
|
||||
|
||||
class TestAuthorizer extends AclAuthorizer {
|
||||
override def authorize(requestContext: AuthorizableRequestContext, actions: util.List[Action]): util.List[AuthorizationResult] = {
|
||||
actions.asScala.map { _ =>
|
||||
if (requestContext.requestType == ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS.id && requestContext.principal == UnauthorizedPrincipal)
|
||||
AuthorizationResult.DENIED
|
||||
else
|
||||
AuthorizationResult.ALLOWED
|
||||
}.asJava
|
||||
}
|
||||
}
|
||||
|
||||
class TestPrincipalBuilderReturningAuthorized extends KafkaPrincipalBuilder {
|
||||
override def build(context: AuthenticationContext): KafkaPrincipal = {
|
||||
AuthorizedPrincipal
|
||||
}
|
||||
}
|
||||
|
||||
class TestPrincipalBuilderReturningUnauthorized extends KafkaPrincipalBuilder {
|
||||
override def build(context: AuthenticationContext): KafkaPrincipal = {
|
||||
UnauthorizedPrincipal
|
||||
}
|
||||
}
|
||||
}
|
|
@ -554,6 +554,12 @@ class RequestQuotaTest extends BaseRequestTest {
|
|||
case ApiKeys.ALTER_CLIENT_QUOTAS =>
|
||||
new AlterClientQuotasRequest.Builder(List.empty.asJava, false)
|
||||
|
||||
case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS =>
|
||||
new DescribeUserScramCredentialsRequest.Builder(new DescribeUserScramCredentialsRequestData())
|
||||
|
||||
case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS =>
|
||||
new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData())
|
||||
|
||||
case _ =>
|
||||
throw new IllegalArgumentException("Unsupported API key " + apiKey)
|
||||
}
|
||||
|
|
|
@ -16,12 +16,12 @@
|
|||
*/
|
||||
package kafka.utils
|
||||
|
||||
import java.io.{File, BufferedWriter, FileWriter}
|
||||
import java.io.{BufferedWriter, File, FileWriter}
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.Seq
|
||||
|
||||
import kafka.server.KafkaConfig
|
||||
import org.apache.kafka.clients.admin.ScramMechanism
|
||||
import org.apache.kafka.common.utils.Java
|
||||
|
||||
object JaasTestUtils {
|
||||
|
@ -169,6 +169,17 @@ 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 = {
|
||||
if (ScramMechanism.fromMechanismName(mechanism) == ScramMechanism.UNKNOWN) {
|
||||
throw new IllegalArgumentException("Unsupported SCRAM mechanism " + mechanism)
|
||||
}
|
||||
ScramLoginModule(
|
||||
scramUser,
|
||||
scramPassword
|
||||
).toString
|
||||
}
|
||||
|
||||
// Returns the dynamic configuration, using credentials for user #1
|
||||
def clientLoginModule(mechanism: String, keytabLocation: Option[File], serviceName: String = serviceName): String =
|
||||
kafkaClientModule(mechanism, keytabLocation, KafkaClientPrincipal, KafkaPlainUser, KafkaPlainPassword, KafkaScramUser, KafkaScramPassword, KafkaOAuthBearerUser, serviceName).toString
|
||||
|
@ -210,14 +221,18 @@ object JaasTestUtils {
|
|||
KafkaPlainUser -> KafkaPlainPassword,
|
||||
KafkaPlainUser2 -> KafkaPlainPassword2
|
||||
))
|
||||
case "SCRAM-SHA-256" | "SCRAM-SHA-512" =>
|
||||
ScramLoginModule(
|
||||
KafkaScramAdmin,
|
||||
KafkaScramAdminPassword,
|
||||
debug = false)
|
||||
case "OAUTHBEARER" =>
|
||||
OAuthBearerLoginModule(KafkaOAuthBearerAdmin)
|
||||
case mechanism => throw new IllegalArgumentException("Unsupported server mechanism " + mechanism)
|
||||
case mechanism => {
|
||||
if (ScramMechanism.fromMechanismName(mechanism) != ScramMechanism.UNKNOWN) {
|
||||
ScramLoginModule(
|
||||
KafkaScramAdmin,
|
||||
KafkaScramAdminPassword,
|
||||
debug = false)
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unsupported server mechanism " + mechanism)
|
||||
}
|
||||
}
|
||||
}
|
||||
JaasSection(contextName, modules)
|
||||
}
|
||||
|
@ -243,16 +258,20 @@ object JaasTestUtils {
|
|||
plainUser,
|
||||
plainPassword
|
||||
)
|
||||
case "SCRAM-SHA-256" | "SCRAM-SHA-512" =>
|
||||
ScramLoginModule(
|
||||
scramUser,
|
||||
scramPassword
|
||||
)
|
||||
case "OAUTHBEARER" =>
|
||||
OAuthBearerLoginModule(
|
||||
oauthBearerUser
|
||||
)
|
||||
case mechanism => throw new IllegalArgumentException("Unsupported client mechanism " + mechanism)
|
||||
case mechanism => {
|
||||
if (ScramMechanism.fromMechanismName(mechanism) != ScramMechanism.UNKNOWN) {
|
||||
ScramLoginModule(
|
||||
scramUser,
|
||||
scramPassword
|
||||
)
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unsupported client mechanism " + mechanism)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -372,7 +372,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
node.account.mkdirs(KafkaService.PERSISTENT_ROOT)
|
||||
|
||||
self.security_config.setup_node(node)
|
||||
self.security_config.setup_credentials(node, self.path, self.zk_connect_setting(), broker=True)
|
||||
self.security_config.maybe_setup_broker_scram_credentials(node, self.path, "--zookeeper %s %s" % (self.zk_connect_setting(), self.zk.zkTlsConfigFileOption()))
|
||||
|
||||
prop_file = self.prop_file(node)
|
||||
self.logger.info("kafka.properties:")
|
||||
|
@ -391,7 +391,9 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
# Credentials for inter-broker communication are created before starting Kafka.
|
||||
# Client credentials are created after starting Kafka so that both loading of
|
||||
# existing credentials from ZK and dynamic update of credentials in Kafka are tested.
|
||||
self.security_config.setup_credentials(node, self.path, self.zk_connect_setting(), broker=False)
|
||||
# We use the admin client and connect as the broker user when creating the client (non-broker) credentials
|
||||
# if Kafka supports KIP-554, otherwise we use ZooKeeper.
|
||||
self.security_config.maybe_setup_client_scram_credentials(node, self.path, self._connect_setting_kafka_configs_scram(node))
|
||||
|
||||
self.start_jmx_tool(self.idx(node), node)
|
||||
if len(self.pids(node)) == 0:
|
||||
|
@ -575,10 +577,21 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
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),
|
||||
self.security_config.client_config())
|
||||
else:
|
||||
return "--zookeeper %s %s" % self.zk_connect_setting(), self.zk.zkTlsConfigFileOption()
|
||||
return "--zookeeper %s %s" % (self.zk_connect_setting(), self.zk.zkTlsConfigFileOption())
|
||||
|
||||
def _connect_setting_kafka_configs_scram(self, node):
|
||||
# Use this for kafka-configs when operating on User SCRAM Credentials
|
||||
if node.version.kafka_configs_command_uses_bootstrap_server_scram():
|
||||
return "--bootstrap-server %s --command-config <(echo '%s')" %\
|
||||
(self.bootstrap_servers(self.security_protocol),
|
||||
self.security_config.client_config(use_inter_broker_mechanism_for_client = True))
|
||||
else:
|
||||
return "--zookeeper %s %s" % (self.zk_connect_setting(), self.zk.zkTlsConfigFileOption())
|
||||
|
||||
def parse_describe_topic(self, topic_description):
|
||||
"""Parse output of kafka-topics.sh --describe (or describe_topic() method above), which is a string of form
|
||||
|
|
|
@ -134,6 +134,8 @@ class SecurityConfig(TemplateRenderer):
|
|||
ZK_CLIENT_TLS_ENCRYPT_ONLY_CONFIG_PATH = "/mnt/security/zk_client_tls_encrypt_only_config.properties"
|
||||
ZK_CLIENT_MUTUAL_AUTH_CONFIG_PATH = "/mnt/security/zk_client_mutual_auth_config.properties"
|
||||
JAAS_CONF_PATH = "/mnt/security/jaas.conf"
|
||||
# allows admin client to connect with broker credentials to create User SCRAM credentials
|
||||
ADMIN_CLIENT_AS_BROKER_JAAS_CONF_PATH = "/mnt/security/admin_client_as_broker_jaas.conf"
|
||||
KRB5CONF_PATH = "/mnt/security/krb5.conf"
|
||||
KEYTAB_PATH = "/mnt/security/keytab"
|
||||
|
||||
|
@ -196,15 +198,20 @@ class SecurityConfig(TemplateRenderer):
|
|||
self.properties.update(self.listener_security_config.client_listener_overrides)
|
||||
self.jaas_override_variables = jaas_override_variables or {}
|
||||
|
||||
def client_config(self, template_props="", node=None, jaas_override_variables=None):
|
||||
def client_config(self, template_props="", node=None, jaas_override_variables=None,
|
||||
use_inter_broker_mechanism_for_client = False):
|
||||
# If node is not specified, use static jaas config which will be created later.
|
||||
# Otherwise use static JAAS configuration files with SASL_SSL and sasl.jaas.config
|
||||
# property with SASL_PLAINTEXT so that both code paths are tested by existing tests.
|
||||
# Note that this is an artibtrary choice and it is possible to run all tests with
|
||||
# either static or dynamic jaas config files if required.
|
||||
static_jaas_conf = node is None or (self.has_sasl and self.has_ssl)
|
||||
if use_inter_broker_mechanism_for_client:
|
||||
client_sasl_mechanism_to_use = self.interbroker_sasl_mechanism
|
||||
else:
|
||||
client_sasl_mechanism_to_use = self.client_sasl_mechanism
|
||||
return SecurityConfig(self.context, self.security_protocol,
|
||||
client_sasl_mechanism=self.client_sasl_mechanism,
|
||||
client_sasl_mechanism=client_sasl_mechanism_to_use,
|
||||
template_props=template_props,
|
||||
static_jaas_conf=static_jaas_conf,
|
||||
jaas_override_variables=jaas_override_variables,
|
||||
|
@ -248,6 +255,10 @@ class SecurityConfig(TemplateRenderer):
|
|||
|
||||
if self.static_jaas_conf:
|
||||
node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf)
|
||||
node.account.create_file(SecurityConfig.ADMIN_CLIENT_AS_BROKER_JAAS_CONF_PATH,
|
||||
self.render_jaas_config("admin_client_as_broker_jaas.conf",
|
||||
{'SecurityConfig': SecurityConfig}))
|
||||
|
||||
elif 'sasl.jaas.config' not in self.properties:
|
||||
self.properties['sasl.jaas.config'] = jaas_conf.replace("\n", " \\\n")
|
||||
if self.has_sasl_kerberos:
|
||||
|
@ -276,18 +287,19 @@ class SecurityConfig(TemplateRenderer):
|
|||
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,
|
||||
SecurityConfig.SCRAM_BROKER_USER, SecurityConfig.SCRAM_BROKER_PASSWORD)
|
||||
else:
|
||||
self.maybe_create_scram_credentials(node, zk_connect, path, self.client_sasl_mechanism,
|
||||
SecurityConfig.SCRAM_CLIENT_USER, SecurityConfig.SCRAM_CLIENT_PASSWORD)
|
||||
def maybe_setup_broker_scram_credentials(self, node, path, connect):
|
||||
self.maybe_create_scram_credentials(node, connect, path, self.interbroker_sasl_mechanism,
|
||||
SecurityConfig.SCRAM_BROKER_USER, SecurityConfig.SCRAM_BROKER_PASSWORD)
|
||||
|
||||
def maybe_create_scram_credentials(self, node, zk_connect, path, mechanism, user_name, password):
|
||||
def maybe_setup_client_scram_credentials(self, node, path, connect):
|
||||
self.maybe_create_scram_credentials(node, connect, path, self.client_sasl_mechanism,
|
||||
SecurityConfig.SCRAM_CLIENT_USER, SecurityConfig.SCRAM_CLIENT_PASSWORD,
|
||||
self.export_kafka_opts_for_admin_client_as_broker())
|
||||
|
||||
def maybe_create_scram_credentials(self, node, connect, path, mechanism, user_name, password, kafka_opts_for_admin_client_as_broker = ""):
|
||||
if self.has_sasl and self.is_sasl_scram(mechanism):
|
||||
cmd = "%s --zookeeper %s --entity-name %s --entity-type users --alter --add-config %s=[password=%s]" % \
|
||||
(path.script("kafka-configs.sh", node), zk_connect,
|
||||
cmd = "%s %s %s --entity-name %s --entity-type users --alter --add-config %s=[password=%s]" % \
|
||||
(kafka_opts_for_admin_client_as_broker, path.script("kafka-configs.sh", node), connect,
|
||||
user_name, mechanism, password)
|
||||
node.account.ssh(cmd)
|
||||
|
||||
|
@ -350,6 +362,14 @@ class SecurityConfig(TemplateRenderer):
|
|||
else:
|
||||
return ""
|
||||
|
||||
def export_kafka_opts_for_admin_client_as_broker(self):
|
||||
if self.has_sasl and self.static_jaas_conf:
|
||||
kafka_opts_to_use = "\"-Djava.security.auth.login.config=%s -Djava.security.krb5.conf=%s\""\
|
||||
% (SecurityConfig.ADMIN_CLIENT_AS_BROKER_JAAS_CONF_PATH, SecurityConfig.KRB5CONF_PATH)
|
||||
else:
|
||||
kafka_opts_to_use = self.kafka_opts
|
||||
return "export KAFKA_OPTS=%s;" % kafka_opts_to_use
|
||||
|
||||
def props(self, prefix=''):
|
||||
"""
|
||||
Return properties as string with line separators, optionally with a prefix.
|
||||
|
|
|
@ -0,0 +1,19 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
|
||||
KafkaClient {
|
||||
org.apache.kafka.common.security.scram.ScramLoginModule required
|
||||
username="{{ SecurityConfig.SCRAM_BROKER_USER }}"
|
||||
password="{{ SecurityConfig.SCRAM_BROKER_PASSWORD }}";
|
||||
};
|
|
@ -78,8 +78,8 @@ class QuotaConfig(object):
|
|||
|
||||
def configure_quota(self, kafka, producer_byte_rate, consumer_byte_rate, entity_args):
|
||||
node = kafka.nodes[0]
|
||||
cmd = "%s --zookeeper %s --alter --add-config producer_byte_rate=%d,consumer_byte_rate=%d" % \
|
||||
(kafka.path.script("kafka-configs.sh", node), kafka.zk_connect_setting(), producer_byte_rate, consumer_byte_rate)
|
||||
cmd = "%s %s --alter --add-config producer_byte_rate=%d,consumer_byte_rate=%d" % \
|
||||
(kafka.path.script("kafka-configs.sh", node), kafka._connect_setting_kafka_configs(node), producer_byte_rate, consumer_byte_rate)
|
||||
cmd += " --entity-type " + entity_args[0] + self.entity_name_opt(entity_args[1])
|
||||
if len(entity_args) > 2:
|
||||
cmd += " --entity-type " + entity_args[2] + self.entity_name_opt(entity_args[3])
|
||||
|
|
|
@ -51,7 +51,7 @@ class TestUpgrade(ProduceConsumeValidateTest):
|
|||
self.logger.info("Upgrade ZooKeeper from %s to %s" % (str(self.zk.nodes[0].version), str(DEV_BRANCH)))
|
||||
self.zk.set_version(DEV_BRANCH)
|
||||
self.zk.restart_cluster()
|
||||
# Confirm we have a successful ZoKeeper upgrade by describing the topic.
|
||||
# Confirm we have a successful ZooKeeper upgrade by describing the topic.
|
||||
# Not trying to detect a problem here leads to failure in the ensuing Kafka roll, which would be a less
|
||||
# intuitive failure than seeing a problem here, so detect ZooKeeper upgrade problems before involving Kafka.
|
||||
self.zk.describe(self.topic)
|
||||
|
|
|
@ -57,14 +57,19 @@ class KafkaVersion(LooseVersion):
|
|||
|
||||
def supports_tls_to_zookeeper(self):
|
||||
# indicate if KIP-515 is available
|
||||
return self > LATEST_2_4
|
||||
return self >= V_2_5_0
|
||||
|
||||
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 >= V_2_7_0
|
||||
|
||||
def get_version(node=None):
|
||||
"""Return the version attached to the given node.
|
||||
Default to DEV_BRANCH if node or node.version is undefined (aka None)
|
||||
|
@ -158,3 +163,7 @@ LATEST_2_5 = V_2_5_1
|
|||
# 2.6.x versions
|
||||
V_2_6_0 = KafkaVersion("2.6.0")
|
||||
LATEST_2_6 = V_2_6_0
|
||||
|
||||
# 2.7.x versions
|
||||
V_2_7_0 = KafkaVersion("2.7.0")
|
||||
LATEST_2_7 = V_2_7_0
|
||||
|
|
Loading…
Reference in New Issue