|
40 | 40 | ConsumerGroupDescription,
|
41 | 41 | MemberAssignment,
|
42 | 42 | MemberDescription)
|
| 43 | +from ._scram import (UserScramCredentialAlteration, # noqa: F401 |
| 44 | + UserScramCredentialUpsertion, |
| 45 | + UserScramCredentialDeletion, |
| 46 | + ScramCredentialInfo, |
| 47 | + ScramMechanism, |
| 48 | + UserScramCredentialsDescription) |
| 49 | + |
43 | 50 | from ..cimpl import (KafkaException, # noqa: F401
|
44 | 51 | KafkaError,
|
45 | 52 | _AdminClientImpl,
|
|
65 | 72 | from confluent_kafka import ConsumerGroupState \
|
66 | 73 | as _ConsumerGroupState
|
67 | 74 |
|
| 75 | + |
68 | 76 | try:
|
69 | 77 | string_type = basestring
|
70 | 78 | except NameError:
|
@@ -235,6 +243,28 @@ def _make_acls_result(f, futmap):
|
235 | 243 | for resource, fut in futmap.items():
|
236 | 244 | fut.set_exception(e)
|
237 | 245 |
|
| 246 | + @staticmethod |
| 247 | + def _make_user_scram_credentials_result(f, futmap): |
| 248 | + try: |
| 249 | + results = f.result() |
| 250 | + len_results = len(results) |
| 251 | + len_futures = len(futmap) |
| 252 | + if len(results) != len_futures: |
| 253 | + raise RuntimeError( |
| 254 | + f"Results length {len_results} is different from future-map length {len_futures}") |
| 255 | + for username, value in results.items(): |
| 256 | + fut = futmap.get(username, None) |
| 257 | + if fut is None: |
| 258 | + raise RuntimeError( |
| 259 | + f"username {username} not found in future-map: {futmap}") |
| 260 | + if isinstance(value, KafkaError): |
| 261 | + fut.set_exception(KafkaException(value)) |
| 262 | + else: |
| 263 | + fut.set_result(value) |
| 264 | + except Exception as e: |
| 265 | + for _, fut in futmap.items(): |
| 266 | + fut.set_exception(e) |
| 267 | + |
238 | 268 | @staticmethod
|
239 | 269 | def _create_future():
|
240 | 270 | f = concurrent.futures.Future()
|
@@ -366,6 +396,59 @@ def _check_alter_consumer_group_offsets_request(request):
|
366 | 396 | raise ValueError(
|
367 | 397 | "Element of 'topic_partitions' must not have negative value for 'offset' field")
|
368 | 398 |
|
| 399 | + @staticmethod |
| 400 | + def _check_describe_user_scram_credentials_request(users): |
| 401 | + if not isinstance(users, list): |
| 402 | + raise TypeError("Expected input to be list of String") |
| 403 | + for user in users: |
| 404 | + if not isinstance(user, string_type): |
| 405 | + raise TypeError("Each value should be a string") |
| 406 | + if not user: |
| 407 | + raise ValueError("'user' cannot be empty") |
| 408 | + |
| 409 | + @staticmethod |
| 410 | + def _check_alter_user_scram_credentials_request(alterations): |
| 411 | + if not isinstance(alterations, list): |
| 412 | + raise TypeError("Expected input to be list") |
| 413 | + if len(alterations) == 0: |
| 414 | + raise ValueError("Expected at least one alteration") |
| 415 | + for alteration in alterations: |
| 416 | + if not isinstance(alteration, UserScramCredentialAlteration): |
| 417 | + raise TypeError("Expected each element of list to be subclass of UserScramCredentialAlteration") |
| 418 | + if alteration.user is None: |
| 419 | + raise TypeError("'user' cannot be None") |
| 420 | + if not isinstance(alteration.user, string_type): |
| 421 | + raise TypeError("'user' must be a string") |
| 422 | + if not alteration.user: |
| 423 | + raise ValueError("'user' cannot be empty") |
| 424 | + |
| 425 | + if isinstance(alteration, UserScramCredentialUpsertion): |
| 426 | + if alteration.password is None: |
| 427 | + raise TypeError("'password' cannot be None") |
| 428 | + if not isinstance(alteration.password, bytes): |
| 429 | + raise TypeError("'password' must be bytes") |
| 430 | + if not alteration.password: |
| 431 | + raise ValueError("'password' cannot be empty") |
| 432 | + |
| 433 | + if alteration.salt is not None and not alteration.salt: |
| 434 | + raise ValueError("'salt' can be None but cannot be empty") |
| 435 | + if alteration.salt and not isinstance(alteration.salt, bytes): |
| 436 | + raise TypeError("'salt' must be bytes") |
| 437 | + |
| 438 | + if not isinstance(alteration.scram_credential_info, ScramCredentialInfo): |
| 439 | + raise TypeError("Expected credential_info to be ScramCredentialInfo Type") |
| 440 | + if alteration.scram_credential_info.iterations < 1: |
| 441 | + raise ValueError("Iterations should be positive") |
| 442 | + if not isinstance(alteration.scram_credential_info.mechanism, ScramMechanism): |
| 443 | + raise TypeError("Expected the mechanism to be ScramMechanism Type") |
| 444 | + elif isinstance(alteration, UserScramCredentialDeletion): |
| 445 | + if not isinstance(alteration.mechanism, ScramMechanism): |
| 446 | + raise TypeError("Expected the mechanism to be ScramMechanism Type") |
| 447 | + else: |
| 448 | + raise TypeError("Expected each element of list 'alterations' " + |
| 449 | + "to be either a UserScramCredentialUpsertion or a " + |
| 450 | + "UserScramCredentialDeletion") |
| 451 | + |
369 | 452 | def create_topics(self, new_topics, **kwargs):
|
370 | 453 | """
|
371 | 454 | Create one or more new topics.
|
@@ -871,3 +954,61 @@ def set_sasl_credentials(self, username, password):
|
871 | 954 | :raises TypeException: Invalid input.
|
872 | 955 | """
|
873 | 956 | super(AdminClient, self).set_sasl_credentials(username, password)
|
| 957 | + |
| 958 | + def describe_user_scram_credentials(self, users, **kwargs): |
| 959 | + """ |
| 960 | + Describe user SASL/SCRAM credentials. |
| 961 | +
|
| 962 | + :param list(str) users: List of user names to describe. |
| 963 | + Duplicate users aren't allowed. |
| 964 | + :param float request_timeout: The overall request timeout in seconds, |
| 965 | + including broker lookup, request transmission, operation time |
| 966 | + on broker, and response. Default: `socket.timeout.ms*1000.0` |
| 967 | +
|
| 968 | + :returns: A dict of futures keyed by user name. |
| 969 | + The future result() method returns the |
| 970 | + :class:`UserScramCredentialsDescription` or |
| 971 | + raises KafkaException |
| 972 | +
|
| 973 | + :rtype: dict[str, future] |
| 974 | +
|
| 975 | + :raises TypeError: Invalid input type. |
| 976 | + :raises ValueError: Invalid input value. |
| 977 | + """ |
| 978 | + AdminClient._check_describe_user_scram_credentials_request(users) |
| 979 | + |
| 980 | + f, futmap = AdminClient._make_futures_v2(users, None, |
| 981 | + AdminClient._make_user_scram_credentials_result) |
| 982 | + |
| 983 | + super(AdminClient, self).describe_user_scram_credentials(users, f, **kwargs) |
| 984 | + |
| 985 | + return futmap |
| 986 | + |
| 987 | + def alter_user_scram_credentials(self, alterations, **kwargs): |
| 988 | + """ |
| 989 | + Alter user SASL/SCRAM credentials. |
| 990 | +
|
| 991 | + :param list(UserScramCredentialAlteration) alterations: List of |
| 992 | + :class:`UserScramCredentialAlteration` to apply. |
| 993 | + The pair (user, mechanism) must be unique among alterations. |
| 994 | + :param float request_timeout: The overall request timeout in seconds, |
| 995 | + including broker lookup, request transmission, operation time |
| 996 | + on broker, and response. Default: `socket.timeout.ms*1000.0` |
| 997 | +
|
| 998 | + :returns: A dict of futures keyed by user name. |
| 999 | + The future result() method returns None or |
| 1000 | + raises KafkaException |
| 1001 | +
|
| 1002 | + :rtype: dict[str, future] |
| 1003 | +
|
| 1004 | + :raises TypeError: Invalid input type. |
| 1005 | + :raises ValueError: Invalid input value. |
| 1006 | + """ |
| 1007 | + AdminClient._check_alter_user_scram_credentials_request(alterations) |
| 1008 | + |
| 1009 | + f, futmap = AdminClient._make_futures_v2(set([alteration.user for alteration in alterations]), None, |
| 1010 | + AdminClient._make_user_scram_credentials_result) |
| 1011 | + |
| 1012 | + super(AdminClient, self).alter_user_scram_credentials(alterations, f, **kwargs) |
| 1013 | + |
| 1014 | + return futmap |
0 commit comments