Skip to content

Commit bb0f00c

Browse files
committed
Topic Admin API
1 parent f6cda94 commit bb0f00c

File tree

8 files changed

+2770
-14
lines changed

8 files changed

+2770
-14
lines changed

confluent_kafka/__init__.py

Lines changed: 185 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,9 @@
55
Message,
66
Producer,
77
TopicPartition,
8+
AdminClientImpl,
9+
NewTopic,
10+
NewPartitions,
811
libversion,
912
version,
1013
TIMESTAMP_NOT_AVAILABLE,
@@ -13,7 +16,188 @@
1316
OFFSET_BEGINNING,
1417
OFFSET_END,
1518
OFFSET_STORED,
16-
OFFSET_INVALID)
19+
OFFSET_INVALID,
20+
CONFIG_SOURCE_UNKNOWN_CONFIG,
21+
CONFIG_SOURCE_DYNAMIC_TOPIC_CONFIG,
22+
CONFIG_SOURCE_DYNAMIC_BROKER_CONFIG,
23+
CONFIG_SOURCE_DYNAMIC_DEFAULT_BROKER_CONFIG,
24+
CONFIG_SOURCE_STATIC_BROKER_CONFIG,
25+
CONFIG_SOURCE_DEFAULT_CONFIG,
26+
RESOURCE_UNKNOWN,
27+
RESOURCE_ANY,
28+
RESOURCE_TOPIC,
29+
RESOURCE_GROUP,
30+
RESOURCE_BROKER)
1731

32+
import concurrent.futures
1833

1934
__version__ = version()[0]
35+
36+
37+
class ConfigEntry(object):
38+
"""
39+
:py:const:`CONFIG_SOURCE_UNKNOWN_CONFIG`,
40+
:py:const:`CONFIG_SOURCE_DYNAMIC_TOPIC_CONFIG`,
41+
:py:const:`CONFIG_SOURCE_DYNAMIC_BROKER_CONFIG`,
42+
:py:const:`CONFIG_SOURCE_DYNAMIC_DEFAULT_BROKER`,
43+
:py:const:`CONFIG_SOURCE_STATIC_BROKER_CONFIG`,
44+
:py:const:`CONFIG_SOURCE_DEFAULT_CONFIG`
45+
46+
"""
47+
48+
source_name_by_type = {
49+
CONFIG_SOURCE_UNKNOWN_CONFIG: 'UNKNOWN_CONFIG',
50+
CONFIG_SOURCE_DYNAMIC_TOPIC_CONFIG: 'DYNAMIC_TOPIC_CONFIG',
51+
CONFIG_SOURCE_DYNAMIC_BROKER_CONFIG: 'DYNAMIC_BROKER_CONFIG',
52+
CONFIG_SOURCE_DYNAMIC_DEFAULT_BROKER_CONFIG: 'DYNAMIC_DEFAULT_BROKER_CONFIG',
53+
CONFIG_SOURCE_STATIC_BROKER_CONFIG: 'STATIC_BROKER_CONFIG',
54+
CONFIG_SOURCE_DEFAULT_CONFIG: 'DEFAULT_CONFIG'
55+
}
56+
57+
def __init__(self, name, value,
58+
source=CONFIG_SOURCE_UNKNOWN_CONFIG,
59+
is_read_only=False,
60+
is_default=False,
61+
is_sensitive=False,
62+
is_synonym=False,
63+
synonyms=[]):
64+
super(ConfigEntry, self).__init__()
65+
self.name = name
66+
self.value = value
67+
self.source = source
68+
self.is_read_only = bool(is_read_only)
69+
self.is_default = bool(is_default)
70+
self.is_sensitive = bool(is_sensitive)
71+
self.is_synonym = bool(is_synonym)
72+
self.synonyms = synonyms
73+
74+
def __repr__(self):
75+
return "ConfigEntry(%s=\"%s\")" % (self.name, self.value)
76+
77+
def __str__(self):
78+
return "%s=\"%s\"" % (self.name, self.value)
79+
80+
@classmethod
81+
def config_source_to_str(cls, source):
82+
"""Return string representation of a config source."""
83+
return ConfigEntry.source_name_by_type.get(source, '%d?' % source)
84+
85+
86+
class ConfigResource(object):
87+
res_name_by_type = {RESOURCE_UNKNOWN: 'unknown',
88+
RESOURCE_ANY: 'any',
89+
RESOURCE_TOPIC: 'topic',
90+
RESOURCE_GROUP: 'group',
91+
RESOURCE_BROKER: 'broker'}
92+
res_type_by_name = {v: k for k, v in res_name_by_type.items()}
93+
94+
def __init__(self, restype, name, configs=None, error=None):
95+
"""
96+
:param: restype int: Resource type, see the RESOURCE_ constants below.
97+
:param: name str: Resource name, depending on restype.
98+
For RESOURCE_BROKER the resource name is the broker id.
99+
100+
:const:`RESOURCE_ANY` (used for lookups),
101+
:const:`RESOURCE_TOPIC`,
102+
:const:`RESOURCE_GROUP`,
103+
:const:`RESOURCE_BROKER`
104+
"""
105+
super(ConfigResource, self).__init__()
106+
107+
if type(name) != str:
108+
raise ValueError("Resource name must be a string")
109+
110+
if type(restype) == str:
111+
if restype.lower() not in self.res_type_by_name:
112+
raise ValueError("Unknown resource type \"%s\": should be a RESOURCE_.. constant or one of %s" %
113+
(restype, ",".join(self.res_type_by_name.keys())))
114+
restype = self.res_type_by_name[restype]
115+
self.restype = restype
116+
self.name = name
117+
self.set_config_dict = dict()
118+
self.add_config_dict = dict()
119+
self.del_config_dict = dict()
120+
self.configs = configs
121+
self.error = error
122+
123+
def __repr__(self):
124+
if self.error is not None:
125+
return "ConfigResource(%s,%s,%r)" % \
126+
(self.res_name_by_type.get(self.restype, '%d' % self.restype), self.name, self.error)
127+
else:
128+
return "ConfigResource(%s,%s)" % \
129+
(self.res_name_by_type.get(self.restype, '%d' % self.restype), self.name)
130+
131+
def __hash__(self):
132+
return hash((self.restype, self.name))
133+
134+
def __cmp__(self, other):
135+
r = self.restype - other.restype
136+
if r != 0:
137+
return r
138+
return self.name.__cmp__(other.name)
139+
140+
def set_config(self, name, value):
141+
""" Set/Overwrite configuration entry """
142+
self.set_config_dict[name] = value
143+
144+
def add_config(self, name, value):
145+
"""
146+
Append value to configuration entry.
147+
148+
Requires broker version >=2.0.0.
149+
"""
150+
self.add_config_dict[name] = value
151+
152+
def del_config(self, name):
153+
"""
154+
Delete configuration entry, reverting it to the default value.
155+
156+
Requires broker version >=2.0.0.
157+
"""
158+
self.del_config_dict[name] = True
159+
160+
161+
class AdminClient (AdminClientImpl):
162+
def __init__(self, conf):
163+
super(AdminClient, self).__init__(conf)
164+
165+
def create_topics(self, new_topics, **kwargs):
166+
""" FIXME create topics """
167+
168+
f = concurrent.futures.Future()
169+
if not f.set_running_or_notify_cancel():
170+
raise RuntimeError("Future was cancelled prematurely")
171+
return super(AdminClient, self).create_topics(new_topics, f, *kwargs)
172+
173+
def delete_topics(self, topics, **kwargs):
174+
""" FIXME delete topics """
175+
176+
f = concurrent.futures.Future()
177+
if not f.set_running_or_notify_cancel():
178+
raise RuntimeError("Future was cancelled prematurely")
179+
return super(AdminClient, self).delete_topics(topics, f, **kwargs)
180+
181+
def create_partitions(self, topics, **kwargs):
182+
""" FIXME create partitions """
183+
184+
f = concurrent.futures.Future()
185+
if not f.set_running_or_notify_cancel():
186+
raise RuntimeError("Future was cancelled prematurely")
187+
return super(AdminClient, self).create_partitions(topics, f, **kwargs)
188+
189+
def describe_configs(self, resources, **kwargs):
190+
""" FIXME describe configs """
191+
192+
f = concurrent.futures.Future()
193+
if not f.set_running_or_notify_cancel():
194+
raise RuntimeError("Future was cancelled prematurely")
195+
return super(AdminClient, self).describe_configs(resources, f, **kwargs)
196+
197+
def alter_configs(self, resources, **kwargs):
198+
""" FIXME alter configs """
199+
200+
f = concurrent.futures.Future()
201+
if not f.set_running_or_notify_cancel():
202+
raise RuntimeError("Future was cancelled prematurely")
203+
return super(AdminClient, self).alter_configs(resources, f, **kwargs)

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