Skip to content

Commit e4eeaa1

Browse files
committed
Move Admin API to admin sub module, use enums, fix doc strings
1 parent d52858c commit e4eeaa1

File tree

8 files changed

+659
-664
lines changed

8 files changed

+659
-664
lines changed

confluent_kafka/__init__.py

Lines changed: 2 additions & 632 deletions
Large diffs are not rendered by default.

confluent_kafka/admin/__init__.py

Lines changed: 622 additions & 0 deletions
Large diffs are not rendered by default.

confluent_kafka/src/Admin.c

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -742,7 +742,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
742742
* topics are of correct type.
743743
* Since this is not in the fast path we treat ourselves
744744
* to the luxury of looking up this for each call. */
745-
ConfigResource_type = cfl_PyObject_lookup("confluent_kafka",
745+
ConfigResource_type = cfl_PyObject_lookup("confluent_kafka.admin",
746746
"ConfigResource");
747747
if (!ConfigResource_type) {
748748
rd_kafka_AdminOptions_destroy(c_options);
@@ -776,7 +776,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
776776
goto err;
777777
}
778778

779-
if (!cfl_PyObject_GetInt(res, "restype", &restype, 0, 0))
779+
if (!cfl_PyObject_GetInt(res, "restype_int", &restype, 0, 0))
780780
goto err;
781781

782782
if (!cfl_PyObject_GetString(res, "name", &resname, NULL, 0))
@@ -889,7 +889,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
889889
* topics are of correct type.
890890
* Since this is not in the fast path we treat ourselves
891891
* to the luxury of looking up this for each call. */
892-
ConfigResource_type = cfl_PyObject_lookup("confluent_kafka",
892+
ConfigResource_type = cfl_PyObject_lookup("confluent_kafka.admin",
893893
"ConfigResource");
894894
if (!ConfigResource_type) {
895895
rd_kafka_AdminOptions_destroy(c_options);
@@ -924,7 +924,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
924924
goto err;
925925
}
926926

927-
if (!cfl_PyObject_GetInt(res, "restype", &restype, 0, 0))
927+
if (!cfl_PyObject_GetInt(res, "restype_int", &restype, 0, 0))
928928
goto err;
929929

930930
if (!cfl_PyObject_GetString(res, "name", &resname, NULL, 0))
@@ -1243,12 +1243,12 @@ Admin_c_ConfigResource_result_to_py (const rd_kafka_ConfigResource_t **c_resourc
12431243
PyObject *ConfigEntry_type;
12441244
size_t ri;
12451245

1246-
ConfigResource_type = cfl_PyObject_lookup("confluent_kafka",
1246+
ConfigResource_type = cfl_PyObject_lookup("confluent_kafka.admin",
12471247
"ConfigResource");
12481248
if (!ConfigResource_type)
12491249
return NULL;
12501250

1251-
ConfigEntry_type = cfl_PyObject_lookup("confluent_kafka",
1251+
ConfigEntry_type = cfl_PyObject_lookup("confluent_kafka.admin",
12521252
"ConfigEntry");
12531253
if (!ConfigEntry_type) {
12541254
Py_DECREF(ConfigResource_type);

confluent_kafka/src/Metadata.c

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ c_partitions_to_py (Handle *self,
3535
PyObject *dict;
3636
int i;
3737

38-
PartitionMetadata_type = cfl_PyObject_lookup("confluent_kafka",
38+
PartitionMetadata_type = cfl_PyObject_lookup("confluent_kafka.admin",
3939
"PartitionMetadata");
4040
if (!PartitionMetadata_type)
4141
return NULL;
@@ -126,7 +126,7 @@ c_topics_to_py (Handle *self, const rd_kafka_metadata_topic_t *c_topics,
126126
PyObject *dict;
127127
int i;
128128

129-
TopicMetadata_type = cfl_PyObject_lookup("confluent_kafka",
129+
TopicMetadata_type = cfl_PyObject_lookup("confluent_kafka.admin",
130130
"TopicMetadata");
131131
if (!TopicMetadata_type)
132132
return NULL;
@@ -200,7 +200,7 @@ static PyObject *c_brokers_to_py (Handle *self,
200200
PyObject *dict;
201201
int i;
202202

203-
BrokerMetadata_type = cfl_PyObject_lookup("confluent_kafka",
203+
BrokerMetadata_type = cfl_PyObject_lookup("confluent_kafka.admin",
204204
"BrokerMetadata");
205205
if (!BrokerMetadata_type)
206206
return NULL;
@@ -264,7 +264,7 @@ c_metadata_to_py (Handle *self, const rd_kafka_metadata_t *metadata) {
264264
char *cluster_id;
265265
#endif
266266

267-
ClusterMetadata_type = cfl_PyObject_lookup("confluent_kafka",
267+
ClusterMetadata_type = cfl_PyObject_lookup("confluent_kafka.admin",
268268
"ClusterMetadata");
269269
if (!ClusterMetadata_type)
270270
return NULL;

examples/adminapi.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,8 @@
1919
# Example Admin clients.
2020
#
2121

22-
from confluent_kafka import AdminClient, NewTopic, NewPartitions, ConfigResource, ConfigEntry, KafkaException
22+
from confluent_kafka.admin import AdminClient, NewTopic, NewPartitions, ConfigResource, ConfigEntry
23+
from confluent_kafka import KafkaException
2324
import sys
2425
import threading
2526
import logging

examples/integration_test.py

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
""" Test script for confluent_kafka module """
2222

2323
import confluent_kafka
24+
from confluent_kafka import admin
2425
import os
2526
import time
2627
import uuid
@@ -1029,7 +1030,7 @@ def verify_topic_metadata(client, exp_topics):
10291030
def verify_admin():
10301031
""" Verify Admin API """
10311032

1032-
a = confluent_kafka.AdminClient({'bootstrap.servers': bootstrap_servers})
1033+
a = admin.AdminClient({'bootstrap.servers': bootstrap_servers})
10331034
our_topic = topic + '_admin_' + str(uuid.uuid4())
10341035
num_partitions = 2
10351036

@@ -1040,10 +1041,10 @@ def verify_admin():
10401041
# Second iteration: create topic.
10411042
#
10421043
for validate in (True, False):
1043-
fs = a.create_topics([confluent_kafka.NewTopic(our_topic,
1044-
num_partitions=num_partitions,
1045-
config=topic_config,
1046-
replication_factor=1)],
1044+
fs = a.create_topics([admin.NewTopic(our_topic,
1045+
num_partitions=num_partitions,
1046+
config=topic_config,
1047+
replication_factor=1)],
10471048
validate_only=validate,
10481049
operation_timeout=10.0)
10491050

@@ -1059,8 +1060,8 @@ def verify_admin():
10591060
# Increase the partition count
10601061
#
10611062
num_partitions += 3
1062-
fs = a.create_partitions([confluent_kafka.NewPartitions(our_topic,
1063-
new_total_count=num_partitions)],
1063+
fs = a.create_partitions([admin.NewPartitions(our_topic,
1064+
new_total_count=num_partitions)],
10641065
operation_timeout=10.0)
10651066

10661067
for topic2, f in fs.items():
@@ -1086,7 +1087,7 @@ def verify_config(expconfig, configs):
10861087
#
10871088
# Get current topic config
10881089
#
1089-
resource = confluent_kafka.ConfigResource(confluent_kafka.RESOURCE_TOPIC, our_topic)
1090+
resource = admin.ConfigResource(admin.RESOURCE_TOPIC, our_topic)
10901091
fs = a.describe_configs([resource])
10911092
configs = fs[resource].result() # will raise exception on failure
10921093

setup.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,8 @@
2727
packages=find_packages(exclude=("tests", "tests.*")),
2828
data_files=[('', ['LICENSE.txt'])],
2929
install_requires=[
30-
'futures;python_version<"3.0"'
30+
'futures;python_version<"3.0"',
31+
'enum34;python_version<"3.0"'
3132
],
3233
extras_require={
3334
'avro': ['fastavro', 'requests', avro]

tests/test_Admin.py

Lines changed: 12 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1,21 +1,21 @@
11
#!/usr/bin/env python
22
import pytest
33

4-
from confluent_kafka import AdminClient, NewTopic, NewPartitions, \
5-
KafkaException, KafkaError, ConfigResource, libversion
4+
from confluent_kafka.admin import AdminClient, NewTopic, NewPartitions, ConfigResource
5+
from confluent_kafka import KafkaException, KafkaError, libversion
66
import confluent_kafka
77
import concurrent.futures
88

99

1010
def test_types():
11-
ConfigResource(confluent_kafka.RESOURCE_BROKER, "2")
11+
ConfigResource(confluent_kafka.admin.RESOURCE_BROKER, "2")
1212
ConfigResource("broker", "2")
13-
ConfigResource(confluent_kafka.RESOURCE_GROUP, "mygroup")
14-
ConfigResource(confluent_kafka.RESOURCE_TOPIC, "")
13+
ConfigResource(confluent_kafka.admin.RESOURCE_GROUP, "mygroup")
14+
ConfigResource(confluent_kafka.admin.RESOURCE_TOPIC, "")
1515
with pytest.raises(ValueError):
1616
ConfigResource("doesnt exist", "hi")
1717
with pytest.raises(ValueError):
18-
ConfigResource(confluent_kafka.RESOURCE_TOPIC, None)
18+
ConfigResource(confluent_kafka.admin.RESOURCE_TOPIC, None)
1919

2020

2121
@pytest.mark.skipif(libversion()[1] < 0x000b0500,
@@ -201,7 +201,7 @@ def test_describe_configs_api():
201201
is no broker configured. """
202202

203203
a = AdminClient({"socket.timeout.ms": 10})
204-
fs = a.describe_configs([ConfigResource(confluent_kafka.RESOURCE_BROKER, "3")])
204+
fs = a.describe_configs([ConfigResource(confluent_kafka.admin.RESOURCE_BROKER, "3")])
205205
# ignore the result
206206

207207
with pytest.raises(Exception):
@@ -214,10 +214,10 @@ def test_describe_configs_api():
214214
a.describe_configs([])
215215

216216
with pytest.raises(ValueError):
217-
a.describe_configs([None, ConfigResource(confluent_kafka.RESOURCE_TOPIC, "mytopic")])
217+
a.describe_configs([None, ConfigResource(confluent_kafka.admin.RESOURCE_TOPIC, "mytopic")])
218218

219-
fs = a.describe_configs([ConfigResource(confluent_kafka.RESOURCE_TOPIC, "mytopic"),
220-
ConfigResource(confluent_kafka.RESOURCE_GROUP, "mygroup")],
219+
fs = a.describe_configs([ConfigResource(confluent_kafka.admin.RESOURCE_TOPIC, "mytopic"),
220+
ConfigResource(confluent_kafka.admin.RESOURCE_GROUP, "mygroup")],
221221
request_timeout=0.123)
222222
with pytest.raises(KafkaException):
223223
for f in concurrent.futures.as_completed(iter(fs.values())):
@@ -231,7 +231,7 @@ def test_alter_configs_api():
231231
is no broker configured. """
232232

233233
a = AdminClient({"socket.timeout.ms": 10})
234-
fs = a.alter_configs([ConfigResource(confluent_kafka.RESOURCE_BROKER, "3",
234+
fs = a.alter_configs([ConfigResource(confluent_kafka.admin.RESOURCE_BROKER, "3",
235235
set_config={"some": "config"})])
236236
# ignore the result
237237

@@ -253,7 +253,7 @@ def test_alter_configs_api():
253253
"and": "this"},
254254
add_config={"add": "this"},
255255
del_config=["this"]),
256-
ConfigResource(confluent_kafka.RESOURCE_GROUP,
256+
ConfigResource(confluent_kafka.admin.RESOURCE_GROUP,
257257
"mygroup")],
258258
request_timeout=0.123)
259259

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