Skip to content

Commit a150b5e

Browse files
mahajanadhityaemasabpranavrthmilindl
authored
Fix describe user scram credentials (confluentinc#1630)
for describing all users --------- Co-authored-by: Emanuele Sabellico <esabellico@confluent.io> Co-authored-by: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Co-authored-by: Milind L <milindl@users.noreply.github.com>
1 parent c391879 commit a150b5e

File tree

9 files changed

+275
-162
lines changed

9 files changed

+275
-162
lines changed

CHANGELOG.md

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -4,12 +4,14 @@
44

55
v2.3.0 is a feature release with the following features, fixes and enhancements:
66

7-
* Add Python 3.12 wheels
8-
* Add support for AdminAPI `DescribeCluster()` and `DescribeTopics()`. (@jainruchir, #1635)
9-
* [KIP-430](https://cwiki.apache.org/confluence/display/KAFKA/KIP-430+-+Return+Authorized+Operations+in+Describe+Responses):
7+
- Add Python 3.12 wheels
8+
- Add support for AdminAPI `DescribeCluster()` and `DescribeTopics()`. (@jainruchir, #1635)
9+
- [KIP-430](https://cwiki.apache.org/confluence/display/KAFKA/KIP-430+-+Return+Authorized+Operations+in+Describe+Responses):
1010
Return authorized operations in Describe Responses. (@jainruchir, #1635)
11-
* Add `Rack` to the `Node` type, so AdminAPI calls can expose racks for brokers
11+
- Add `Rack` to the `Node` type, so AdminAPI calls can expose racks for brokers
1212
(currently, all Describe Responses) (#1635, @jainruchir).
13+
- Fix the Describe User Scram Credentials for Describe all users or empty users list. Please refer to
14+
issue(https://github.com/confluentinc/confluent-kafka-python/issues/1616) for more details (#1630).
1315

1416
confluent-kafka-python is based on librdkafka v2.3.0, see the
1517
[librdkafka release notes](https://github.com/confluentinc/librdkafka/releases/tag/v2.3.0)

examples/adminapi.py

Lines changed: 38 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -688,19 +688,46 @@ def example_describe_user_scram_credentials(a, args):
688688
"""
689689
Describe User Scram Credentials
690690
"""
691-
futmap = a.describe_user_scram_credentials(args)
692-
693-
for username, fut in futmap.items():
694-
print("Username: {}".format(username))
691+
if len(args) == 0:
692+
"""
693+
Case: Describes all user scram credentials
694+
Input: no argument passed or None
695+
Gets a future which result will give a
696+
dict[str, UserScramCredentialsDescription]
697+
or will throw a KafkaException
698+
"""
699+
f = a.describe_user_scram_credentials()
695700
try:
696-
response = fut.result()
697-
for scram_credential_info in response.scram_credential_infos:
698-
print(f" Mechanism: {scram_credential_info.mechanism} " +
699-
f"Iterations: {scram_credential_info.iterations}")
701+
results = f.result()
702+
for username, response in results.items():
703+
print("Username : {}".format(username))
704+
for scram_credential_info in response.scram_credential_infos:
705+
print(f" Mechanism: {scram_credential_info.mechanism} " +
706+
f"Iterations: {scram_credential_info.iterations}")
700707
except KafkaException as e:
701-
print(" Error: {}".format(e))
702-
except Exception as e:
703-
print(f" Unexpected exception: {e}")
708+
print("Failed to describe all user scram credentials : {}".format(e))
709+
except Exception:
710+
raise
711+
else:
712+
"""
713+
Case: Describe specified user scram credentials
714+
Input: users is a list
715+
Gets a dict[str, future] where the result() of
716+
each future will give a UserScramCredentialsDescription
717+
or a KafkaException
718+
"""
719+
futmap = a.describe_user_scram_credentials(args)
720+
for username, fut in futmap.items():
721+
print("Username: {}".format(username))
722+
try:
723+
response = fut.result()
724+
for scram_credential_info in response.scram_credential_infos:
725+
print(f" Mechanism: {scram_credential_info.mechanism} " +
726+
f"Iterations: {scram_credential_info.iterations}")
727+
except KafkaException as e:
728+
print(" Error: {}".format(e))
729+
except Exception:
730+
raise
704731

705732

706733
def example_alter_user_scram_credentials(a, args):

src/confluent_kafka/admin/__init__.py

Lines changed: 49 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -251,28 +251,6 @@ def _make_acls_result(f, futmap):
251251
for resource, fut in futmap.items():
252252
fut.set_exception(e)
253253

254-
@staticmethod
255-
def _make_user_scram_credentials_result(f, futmap):
256-
try:
257-
results = f.result()
258-
len_results = len(results)
259-
len_futures = len(futmap)
260-
if len(results) != len_futures:
261-
raise RuntimeError(
262-
f"Results length {len_results} is different from future-map length {len_futures}")
263-
for username, value in results.items():
264-
fut = futmap.get(username, None)
265-
if fut is None:
266-
raise RuntimeError(
267-
f"username {username} not found in future-map: {futmap}")
268-
if isinstance(value, KafkaError):
269-
fut.set_exception(KafkaException(value))
270-
else:
271-
fut.set_result(value)
272-
except Exception as e:
273-
for _, fut in futmap.items():
274-
fut.set_exception(e)
275-
276254
@staticmethod
277255
def _make_futmap_result_from_list(f, futmap):
278256
try:
@@ -366,6 +344,30 @@ def _make_futures_v2(futmap_keys, class_check, make_result_fn):
366344

367345
return f, futmap
368346

347+
@staticmethod
348+
def _make_single_future_pair():
349+
"""
350+
Create an pair of futures, one for internal usage and one
351+
to use externally, the external one throws a KafkaException if
352+
any of the values in the map returned by the first future is
353+
a KafkaError.
354+
"""
355+
def single_future_result(internal_f, f):
356+
try:
357+
results = internal_f.result()
358+
for _, value in results.items():
359+
if isinstance(value, KafkaError):
360+
f.set_exception(KafkaException(value))
361+
return
362+
f.set_result(results)
363+
except Exception as e:
364+
f.set_exception(e)
365+
366+
f = AdminClient._create_future()
367+
internal_f = AdminClient._create_future()
368+
internal_f.add_done_callback(lambda internal_f: single_future_result(internal_f, f))
369+
return internal_f, f
370+
369371
@staticmethod
370372
def _has_duplicates(items):
371373
return len(set(items)) != len(items)
@@ -449,9 +451,13 @@ def _check_alter_consumer_group_offsets_request(request):
449451

450452
@staticmethod
451453
def _check_describe_user_scram_credentials_request(users):
454+
if users is None:
455+
return
452456
if not isinstance(users, list):
453457
raise TypeError("Expected input to be list of String")
454458
for user in users:
459+
if user is None:
460+
raise TypeError("'user' cannot be None")
455461
if not isinstance(user, string_type):
456462
raise TypeError("Each value should be a string")
457463
if not user:
@@ -1094,34 +1100,41 @@ def set_sasl_credentials(self, username, password):
10941100
"""
10951101
super(AdminClient, self).set_sasl_credentials(username, password)
10961102

1097-
def describe_user_scram_credentials(self, users, **kwargs):
1103+
def describe_user_scram_credentials(self, users=None, **kwargs):
10981104
"""
10991105
Describe user SASL/SCRAM credentials.
11001106
11011107
:param list(str) users: List of user names to describe.
1102-
Duplicate users aren't allowed.
1108+
Duplicate users aren't allowed. Can be None
1109+
to describe all user's credentials.
11031110
:param float request_timeout: The overall request timeout in seconds,
11041111
including broker lookup, request transmission, operation time
11051112
on broker, and response. Default: `socket.timeout.ms*1000.0`
11061113
1107-
:returns: A dict of futures keyed by user name.
1108-
The future result() method returns the
1109-
:class:`UserScramCredentialsDescription` or
1110-
raises KafkaException
1114+
:returns: In case None is passed it returns a single future.
1115+
The future yields a dict[str, UserScramCredentialsDescription]
1116+
or raises a KafkaException
11111117
1112-
:rtype: dict[str, future]
1118+
In case a list of user names is passed, it returns
1119+
a dict[str, future[UserScramCredentialsDescription]].
1120+
The futures yield a :class:`UserScramCredentialsDescription`
1121+
or raise a KafkaException
1122+
1123+
:rtype: Union[future[dict[str, UserScramCredentialsDescription]],
1124+
dict[str, future[UserScramCredentialsDescription]]]
11131125
11141126
:raises TypeError: Invalid input type.
11151127
:raises ValueError: Invalid input value.
11161128
"""
11171129
AdminClient._check_describe_user_scram_credentials_request(users)
11181130

1119-
f, futmap = AdminClient._make_futures_v2(users, None,
1120-
AdminClient._make_user_scram_credentials_result)
1121-
1122-
super(AdminClient, self).describe_user_scram_credentials(users, f, **kwargs)
1123-
1124-
return futmap
1131+
if users is None:
1132+
internal_f, ret_fut = AdminClient._make_single_future_pair()
1133+
else:
1134+
internal_f, ret_fut = AdminClient._make_futures_v2(users, None,
1135+
AdminClient._make_futmap_result)
1136+
super(AdminClient, self).describe_user_scram_credentials(users, internal_f, **kwargs)
1137+
return ret_fut
11251138

11261139
def alter_user_scram_credentials(self, alterations, **kwargs):
11271140
"""
@@ -1146,7 +1159,7 @@ def alter_user_scram_credentials(self, alterations, **kwargs):
11461159
AdminClient._check_alter_user_scram_credentials_request(alterations)
11471160

11481161
f, futmap = AdminClient._make_futures_v2(set([alteration.user for alteration in alterations]), None,
1149-
AdminClient._make_user_scram_credentials_result)
1162+
AdminClient._make_futmap_result)
11501163

11511164
super(AdminClient, self).alter_user_scram_credentials(alterations, f, **kwargs)
11521165
return futmap

src/confluent_kafka/src/Admin.c

Lines changed: 25 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -1809,7 +1809,7 @@ static PyObject *Admin_describe_user_scram_credentials(Handle *self, PyObject *a
18091809
NULL };
18101810
struct Admin_options options = Admin_options_INITIALIZER;
18111811
rd_kafka_AdminOptions_t *c_options = NULL;
1812-
int user_cnt, i;
1812+
int user_cnt = 0, i;
18131813
const char **c_users = NULL;
18141814
rd_kafka_queue_t *rkqu;
18151815
CallState cs;
@@ -1820,7 +1820,7 @@ static PyObject *Admin_describe_user_scram_credentials(Handle *self, PyObject *a
18201820
&options.request_timeout))
18211821
return NULL;
18221822

1823-
if (!PyList_Check(users)) {
1823+
if (users != Py_None && !PyList_Check(users)) {
18241824
PyErr_SetString(PyExc_ValueError,
18251825
"Expected non-empty list of string "
18261826
"objects in 'users' parameter");
@@ -1836,33 +1836,34 @@ static PyObject *Admin_describe_user_scram_credentials(Handle *self, PyObject *a
18361836
* is finished, so we need to keep our own refcount. */
18371837
Py_INCREF(future);
18381838

1839-
user_cnt = (int)PyList_Size(users);
1839+
if (users != Py_None) {
1840+
user_cnt = (int)PyList_Size(users);
1841+
if (user_cnt > 0)
1842+
c_users = malloc(sizeof(char *) * user_cnt);
18401843

1844+
for (i = 0 ; i < user_cnt ; i++) {
1845+
PyObject *user = PyList_GET_ITEM(users, i);
1846+
PyObject *u_user;
1847+
PyObject *uo_user = NULL;
18411848

1842-
c_users = malloc(sizeof(char *) * user_cnt);
1843-
1844-
for (i = 0 ; i < user_cnt ; i++) {
1845-
PyObject *user = PyList_GET_ITEM(users, i);
1846-
PyObject *u_user;
1847-
PyObject *uo_user = NULL;
1849+
if (user == Py_None) {
1850+
PyErr_Format(PyExc_TypeError,
1851+
"User %d in 'users' parameters must not "
1852+
"be None", i);
1853+
goto err;
1854+
}
18481855

1849-
if (user == Py_None) {
1850-
PyErr_Format(PyExc_TypeError,
1851-
"User %d in 'users' parameters must not "
1852-
"be None", i);
1853-
goto err;
1854-
}
1856+
if (!(u_user = cfl_PyObject_Unistr(user))) {
1857+
PyErr_Format(PyExc_ValueError,
1858+
"User %d in 'users' parameters must "
1859+
" be convertible to str", i);
1860+
goto err;
1861+
}
18551862

1856-
if (!(u_user = cfl_PyObject_Unistr(user))) {
1857-
PyErr_Format(PyExc_ValueError,
1858-
"User %d in 'users' parameters must "
1859-
" be convertible to str", i);
1860-
goto err;
1863+
c_users[i] = cfl_PyUnistr_AsUTF8(u_user, &uo_user);
1864+
Py_XDECREF(u_user);
1865+
Py_XDECREF(uo_user);
18611866
}
1862-
1863-
c_users[i] = cfl_PyUnistr_AsUTF8(u_user, &uo_user);
1864-
Py_XDECREF(u_user);
1865-
Py_XDECREF(uo_user);
18661867
}
18671868
/* Use librdkafka's background thread queue to automatically dispatch
18681869
* Admin_background_event_cb() when the admin operation is finished. */

tests/integration/admin/test_basic_operations.py

Lines changed: 2 additions & 85 deletions
Original file line numberDiff line numberDiff line change
@@ -16,15 +16,11 @@
1616
import confluent_kafka
1717
import struct
1818
import time
19-
import pytest
2019
from confluent_kafka import ConsumerGroupTopicPartitions, TopicPartition, ConsumerGroupState
2120
from confluent_kafka.admin import (NewPartitions, ConfigResource,
2221
AclBinding, AclBindingFilter, ResourceType,
23-
ResourcePatternType, AclOperation, AclPermissionType,
24-
UserScramCredentialsDescription, UserScramCredentialUpsertion,
25-
UserScramCredentialDeletion, ScramCredentialInfo,
26-
ScramMechanism)
27-
from confluent_kafka.error import ConsumeError, KafkaException, KafkaError
22+
ResourcePatternType, AclOperation, AclPermissionType)
23+
from confluent_kafka.error import ConsumeError
2824

2925
topic_prefix = "test-topic"
3026

@@ -196,83 +192,6 @@ def verify_consumer_group_offsets_operations(client, our_topic, group_id):
196192
assert topic_partition.offset == 0
197193

198194

199-
def verify_admin_scram(admin_client):
200-
newuser = "non-existent"
201-
newmechanism = ScramMechanism.SCRAM_SHA_256
202-
newiterations = 10000
203-
204-
futmap = admin_client.describe_user_scram_credentials([newuser])
205-
assert isinstance(futmap, dict)
206-
assert len(futmap) == 1
207-
assert newuser in futmap
208-
fut = futmap[newuser]
209-
with pytest.raises(KafkaException) as ex:
210-
result = fut.result()
211-
assert ex.value.args[0] == KafkaError.RESOURCE_NOT_FOUND
212-
213-
futmap = admin_client.alter_user_scram_credentials([UserScramCredentialUpsertion(newuser,
214-
ScramCredentialInfo(newmechanism, newiterations),
215-
b"password", b"salt")])
216-
fut = futmap[newuser]
217-
result = fut.result()
218-
assert result is None
219-
220-
futmap = admin_client.alter_user_scram_credentials([UserScramCredentialUpsertion(
221-
newuser,
222-
ScramCredentialInfo(
223-
ScramMechanism.SCRAM_SHA_256, 10000),
224-
b"password", b"salt"),
225-
UserScramCredentialUpsertion(
226-
newuser,
227-
ScramCredentialInfo(
228-
ScramMechanism.SCRAM_SHA_512, 10000),
229-
b"password")
230-
])
231-
fut = futmap[newuser]
232-
result = fut.result()
233-
assert result is None
234-
235-
futmap = admin_client.alter_user_scram_credentials([UserScramCredentialUpsertion(
236-
newuser,
237-
ScramCredentialInfo(
238-
ScramMechanism.SCRAM_SHA_256, 10000),
239-
b"password", b"salt"),
240-
UserScramCredentialDeletion(
241-
newuser,
242-
ScramMechanism.SCRAM_SHA_512)
243-
])
244-
fut = futmap[newuser]
245-
result = fut.result()
246-
assert result is None
247-
248-
futmap = admin_client.describe_user_scram_credentials([newuser])
249-
assert isinstance(futmap, dict)
250-
assert len(futmap) == 1
251-
assert newuser in futmap
252-
description = futmap[newuser].result()
253-
assert isinstance(description, UserScramCredentialsDescription)
254-
for scram_credential_info in description.scram_credential_infos:
255-
assert ((scram_credential_info.mechanism == newmechanism) and
256-
(scram_credential_info.iterations == newiterations))
257-
258-
futmap = admin_client.alter_user_scram_credentials([UserScramCredentialDeletion(newuser, newmechanism)])
259-
assert isinstance(futmap, dict)
260-
assert len(futmap) == 1
261-
assert newuser in futmap
262-
fut = futmap[newuser]
263-
result = fut.result()
264-
assert result is None
265-
266-
futmap = admin_client.describe_user_scram_credentials([newuser])
267-
assert isinstance(futmap, dict)
268-
assert len(futmap) == 1
269-
assert newuser in futmap
270-
fut = futmap[newuser]
271-
with pytest.raises(KafkaException) as ex:
272-
result = fut.result()
273-
assert ex.value.args[0] == KafkaError.RESOURCE_NOT_FOUND
274-
275-
276195
def test_basic_operations(kafka_cluster):
277196
num_partitions = 2
278197
topic_config = {"compression.type": "gzip"}
@@ -452,5 +371,3 @@ def verify_config(expconfig, configs):
452371

453372
# Verify ACL operations
454373
verify_admin_acls(admin_client, acls_topic, acls_group)
455-
# Verify user SCRAM credentials API
456-
verify_admin_scram(admin_client)

0 commit comments

Comments
 (0)
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy