Skip to content

Commit c87cc7f

Browse files
committed
Change TypeError to ValueError for invalid input
1 parent a08db1c commit c87cc7f

File tree

3 files changed

+52
-51
lines changed

3 files changed

+52
-51
lines changed

confluent_kafka/__init__.py

Lines changed: 18 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -139,16 +139,18 @@ def __init__(self, restype, name,
139139
:param name str: Resource name, depending on restype.
140140
For RESOURCE_BROKER the resource name is the broker id.
141141
:param set_config dict: Configuration to set/overwrite. Dict of str, str.
142-
:param add_config dict: Configuration to add/append. Dict of str, str. Requires broker version >=2.0.
143-
:param del_config list: Configuration to delete/revert to default. List of str. Requires broker version >=2.0.
142+
:param add_config dict: Configuration to add/append. Dict of str, str.
143+
Requires broker version with KIP-248 support.
144+
:param del_config list: Configuration to delete/revert to default. List of str.
145+
Requires broker version with KIP-248 support.
144146
:param described_configs dict: For internal use only.
145147
:param error KafkaError: For internal use only.
146148
147149
When alter_configs(incremental=False) only set_config is permitted,
148150
and any configuration parameter not specified will be reverted to
149151
its default value.
150152
151-
With alter_configs(incremental=True) (requires broker version >=2.0),
153+
With alter_configs(incremental=True) (requires broker version with KIP-248 support)
152154
only the configuration parameters specified through set, add or del
153155
will be modified.
154156
"""
@@ -222,7 +224,7 @@ def add_config(self, name, value):
222224
"""
223225
Append value to configuration entry.
224226
225-
Requires broker version >=2.0.0 and alter_configs(.., incremental=True).
227+
Requires broker version with KIP-248 support and alter_configs(.., incremental=True).
226228
227229
:param name str: Configuration property name
228230
:param value str: Configuration value
@@ -233,7 +235,7 @@ def del_config(self, name):
233235
"""
234236
Delete configuration entry, reverting it to the default value.
235237
236-
Requires broker version >=2.0.0 and alter_configs(.., incremental=True).
238+
Requires broker version with KIP-248 support and alter_configs(.., incremental=True).
237239
238240
:param name str: Configuration property name
239241
"""
@@ -331,7 +333,7 @@ def _make_futures(futmap_keys, class_check, make_result_fn):
331333
futmap = {}
332334
for key in futmap_keys:
333335
if class_check is not None and not isinstance(key, class_check):
334-
raise TypeError("Expected list of {}".format(type(class_check)))
336+
raise ValueError("Expected list of {}".format(type(class_check)))
335337
futmap[key] = concurrent.futures.Future()
336338
if not futmap[key].set_running_or_notify_cancel():
337339
raise RuntimeError("Future was cancelled prematurely")
@@ -492,8 +494,7 @@ def alter_configs(self, resources, **kwargs):
492494
reverting all other configuration for the resource back
493495
to their default values.
494496
Use incremental=True to change the behaviour so that only the
495-
passed configuration is modified.
496-
Requires broker version >=2.0.
497+
passed configuration is modified, requires broker version with KIP-248 support.
497498
498499
:warning: Multiple resources and resource types may be specified,
499500
but at most one resource of type RESOURCE_BROKER is allowed
@@ -508,7 +509,7 @@ def alter_configs(self, resources, **kwargs):
508509
without altering the configuration. Default: False
509510
:param incremental bool: If true, only update the specified configuration
510511
entries, not reverting unspecified configuration.
511-
This requires broker version >=2.0. Default: False
512+
This requires broker version with KIP-248 support. Default: False
512513
513514
:returns: a dict of futures for each resource, keyed by the ConfigResource.
514515
:rtype dict(<ConfigResource, future>):
@@ -531,14 +532,14 @@ class ClusterMetadata (object):
531532
ClusterMetadata as returned by list_topics() contains information
532533
about the Kafka cluster, brokers, and topics.
533534
535+
This class is typically not user instantiated.
536+
534537
:ivar cluster_id: Cluster id string, if supported by broker, else None.
535538
:ivar controller_id: Current controller broker id, or -1.
536539
:ivar brokers: Map of brokers indexed by the int broker id. Value is BrokerMetadata object.
537540
:ivar topics: Map of topics indexed by the topic name. Value is TopicMetadata object.
538541
:ivar orig_broker_id: The broker this metadata originated from.
539542
:ivar orig_broker_name: Broker name/address this metadata originated from.
540-
541-
This class is typically not user instantiated.
542543
"""
543544
def __init__(self):
544545
self.cluster_id = None
@@ -559,11 +560,11 @@ class BrokerMetadata (object):
559560
"""
560561
BrokerMetadata contains information about a Kafka broker.
561562
563+
This class is typically not user instantiated.
564+
562565
:ivar id: Broker id.
563566
:ivar host: Broker hostname.
564567
:ivar port: Broker port.
565-
566-
This class is typically not user instantiated.
567568
"""
568569
def __init__(self):
569570
self.id = -1
@@ -581,11 +582,11 @@ class TopicMetadata (object):
581582
"""
582583
TopicMetadata contains information about a Kafka topic.
583584
585+
This class is typically not user instantiated.
586+
584587
:ivar topic: Topic name.
585588
:ivar partitions: Map of partitions indexed by partition id. Value is PartitionMetadata object.
586589
:ivar error: Topic error, or None. Value is a KafkaError object.
587-
588-
This class is typically not user instantiated.
589590
"""
590591
def __init__(self):
591592
self.topic = None
@@ -606,6 +607,8 @@ class PartitionMetadata (object):
606607
"""
607608
PartitionsMetadata contains information about a Kafka partition.
608609
610+
This class is typically not user instantiated.
611+
609612
:ivar id: Partition id.
610613
:ivar leader: Current leader broker for this partition, or -1.
611614
:ivar replicas: List of replica broker ids for this partition.
@@ -616,8 +619,6 @@ class PartitionMetadata (object):
616619
leader, replicas and isrs may temporarily not be reported
617620
in ClusterMetadata.brokers. Always check the availability
618621
of a broker id in the brokers dict.
619-
620-
This class is typically not user instantiated.
621622
"""
622623
def __init__(self):
623624
self.partition = -1

confluent_kafka/src/Admin.c

Lines changed: 21 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -174,7 +174,7 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
174174
if (!PyList_Check(ra) ||
175175
(int)PyList_Size(ra) < min_count ||
176176
(int)PyList_Size(ra) > max_count) {
177-
PyErr_Format(PyExc_TypeError,
177+
PyErr_Format(PyExc_ValueError,
178178
"replica_assignment must be "
179179
"a list of int lists with an "
180180
"outer size of %s", err_count_desc);
@@ -192,7 +192,7 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
192192
if (!PyList_Check(replicas) ||
193193
(replica_cnt = (size_t)PyList_Size(replicas)) < 1) {
194194
PyErr_Format(
195-
PyExc_TypeError,
195+
PyExc_ValueError,
196196
"replica_assignment must be "
197197
"a list of int lists with an "
198198
"outer size of %s", err_count_desc);
@@ -208,7 +208,7 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
208208

209209
if (!cfl_PyInt_Check(replica)) {
210210
PyErr_Format(
211-
PyExc_TypeError,
211+
PyExc_ValueError,
212212
"replica_assignment must be "
213213
"a list of int lists with an "
214214
"outer size of %s", err_count_desc);
@@ -239,7 +239,7 @@ static int Admin_set_replica_assignment (const char *forApi, void *c_obj,
239239

240240
if (err) {
241241
PyErr_SetString(
242-
PyExc_TypeError, errstr);
242+
PyExc_ValueError, errstr);
243243
return 0;
244244
}
245245
}
@@ -266,7 +266,7 @@ Admin_config_dict_to_c (void *c_obj, PyObject *dict, const char *op_name) {
266266
rd_kafka_resp_err_t err;
267267

268268
if (!(ks = cfl_PyObject_Unistr(ko))) {
269-
PyErr_Format(PyExc_TypeError,
269+
PyErr_Format(PyExc_ValueError,
270270
"expected %s config name to be unicode "
271271
"string", op_name);
272272
return 0;
@@ -281,7 +281,7 @@ Admin_config_dict_to_c (void *c_obj, PyObject *dict, const char *op_name) {
281281
PyObject *vs = NULL, *vs8 = NULL;
282282
if (!(vs = cfl_PyObject_Unistr(vo)) ||
283283
!(v = cfl_PyUnistr_AsUTF8(vs, &vs8))) {
284-
PyErr_Format(PyExc_TypeError,
284+
PyErr_Format(PyExc_ValueError,
285285
"expect %s config value for %s "
286286
"to be unicode string",
287287
op_name, k);
@@ -358,7 +358,7 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
358358
return NULL;
359359

360360
if (!PyList_Check(topics) || (tcnt = (int)PyList_Size(topics)) < 1) {
361-
PyErr_SetString(PyExc_TypeError,
361+
PyErr_SetString(PyExc_ValueError,
362362
"Expected non-empty list of NewTopic objects");
363363
return NULL;
364364
}
@@ -392,7 +392,7 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
392392
if (r == -1)
393393
goto err; /* Exception raised by IsInstance() */
394394
else if (r == 0) {
395-
PyErr_SetString(PyExc_TypeError,
395+
PyErr_SetString(PyExc_ValueError,
396396
"Expected list of NewTopic objects");
397397
goto err;
398398
}
@@ -402,7 +402,7 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
402402
newt->replication_factor,
403403
errstr, sizeof(errstr));
404404
if (!c_objs[i]) {
405-
PyErr_Format(PyExc_TypeError,
405+
PyErr_Format(PyExc_ValueError,
406406
"Invalid NewTopic(%s): %s",
407407
newt->topic, errstr);
408408
i++;
@@ -411,7 +411,7 @@ static PyObject *Admin_create_topics (Handle *self, PyObject *args,
411411

412412
if (newt->replica_assignment) {
413413
if (newt->replication_factor != -1) {
414-
PyErr_SetString(PyExc_TypeError,
414+
PyErr_SetString(PyExc_ValueError,
415415
"replication_factor and "
416416
"replica_assignment are "
417417
"mutually exclusive");
@@ -501,7 +501,7 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
501501
return NULL;
502502

503503
if (!PyList_Check(topics) || (tcnt = (int)PyList_Size(topics)) < 1) {
504-
PyErr_SetString(PyExc_TypeError,
504+
PyErr_SetString(PyExc_ValueError,
505505
"Expected non-empty list of topic strings");
506506
return NULL;
507507
}
@@ -527,7 +527,7 @@ static PyObject *Admin_delete_topics (Handle *self, PyObject *args,
527527

528528
if (topic == Py_None ||
529529
!(utopic = cfl_PyObject_Unistr(topic))) {
530-
PyErr_Format(PyExc_TypeError,
530+
PyErr_Format(PyExc_ValueError,
531531
"Expected list of topic strings, "
532532
"not %s",
533533
((PyTypeObject *)PyObject_Type(topic))->
@@ -605,7 +605,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
605605
return NULL;
606606

607607
if (!PyList_Check(topics) || (tcnt = (int)PyList_Size(topics)) < 1) {
608-
PyErr_SetString(PyExc_TypeError,
608+
PyErr_SetString(PyExc_ValueError,
609609
"Expected non-empty list of "
610610
"NewPartitions objects");
611611
return NULL;
@@ -642,7 +642,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
642642
if (r == -1)
643643
goto err; /* Exception raised by IsInstance() */
644644
else if (r == 0) {
645-
PyErr_SetString(PyExc_TypeError,
645+
PyErr_SetString(PyExc_ValueError,
646646
"Expected list of "
647647
"NewPartitions objects");
648648
goto err;
@@ -652,7 +652,7 @@ static PyObject *Admin_create_partitions (Handle *self, PyObject *args,
652652
newp->new_total_count,
653653
errstr, sizeof(errstr));
654654
if (!c_objs[i]) {
655-
PyErr_Format(PyExc_TypeError,
655+
PyErr_Format(PyExc_ValueError,
656656
"Invalid NewPartitions(%s): %s",
657657
newp->topic, errstr);
658658
goto err;
@@ -732,7 +732,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
732732

733733
if (!PyList_Check(resources) ||
734734
(cnt = (int)PyList_Size(resources)) < 1) {
735-
PyErr_SetString(PyExc_TypeError,
735+
PyErr_SetString(PyExc_ValueError,
736736
"Expected non-empty list of ConfigResource "
737737
"objects");
738738
return NULL;
@@ -775,7 +775,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
775775
if (r == -1)
776776
goto err; /* Exception raised by IsInstance() */
777777
else if (r == 0) {
778-
PyErr_SetString(PyExc_TypeError,
778+
PyErr_SetString(PyExc_ValueError,
779779
"Expected list of "
780780
"ConfigResource objects");
781781
goto err;
@@ -790,7 +790,7 @@ static PyObject *Admin_describe_configs (Handle *self, PyObject *args,
790790
c_objs[i] = rd_kafka_ConfigResource_new(
791791
(rd_kafka_ResourceType_t)restype, resname);
792792
if (!c_objs[i]) {
793-
PyErr_Format(PyExc_TypeError,
793+
PyErr_Format(PyExc_ValueError,
794794
"Invalid ConfigResource(%d,%s)",
795795
restype, resname);
796796
free(resname);
@@ -871,7 +871,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
871871

872872
if (!PyList_Check(resources) ||
873873
(cnt = (int)PyList_Size(resources)) < 1) {
874-
PyErr_SetString(PyExc_TypeError,
874+
PyErr_SetString(PyExc_ValueError,
875875
"Expected non-empty list of ConfigResource "
876876
"objects");
877877
return NULL;
@@ -926,7 +926,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
926926
if (r == -1)
927927
goto err; /* Exception raised by IsInstance() */
928928
else if (r == 0) {
929-
PyErr_SetString(PyExc_TypeError,
929+
PyErr_SetString(PyExc_ValueError,
930930
"Expected list of "
931931
"ConfigResource objects");
932932
goto err;
@@ -941,7 +941,7 @@ static PyObject *Admin_alter_configs (Handle *self, PyObject *args,
941941
c_objs[i] = rd_kafka_ConfigResource_new(
942942
(rd_kafka_ResourceType_t)restype, resname);
943943
if (!c_objs[i]) {
944-
PyErr_Format(PyExc_TypeError,
944+
PyErr_Format(PyExc_ValueError,
945945
"Invalid ConfigResource(%d,%s)",
946946
restype, resname);
947947
free(resname);

tests/test_Admin.py

Lines changed: 13 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -121,16 +121,16 @@ def test_delete_topics_api():
121121
fs = a.delete_topics(["mytopic"])
122122
# ignore the result
123123

124-
with pytest.raises(TypeError):
124+
with pytest.raises(Exception):
125125
a.delete_topics(None)
126126

127-
with pytest.raises(TypeError):
127+
with pytest.raises(Exception):
128128
a.delete_topics("mytopic")
129129

130-
with pytest.raises(TypeError):
130+
with pytest.raises(Exception):
131131
a.delete_topics([])
132132

133-
with pytest.raises(TypeError):
133+
with pytest.raises(ValueError):
134134
a.delete_topics([None, "mytopic"])
135135

136136
fs = a.delete_topics(["mytopic", "other"])
@@ -167,7 +167,7 @@ def test_create_partitions_api():
167167
with pytest.raises(Exception):
168168
a.create_partitions("mytopic")
169169

170-
with pytest.raises(TypeError):
170+
with pytest.raises(Exception):
171171
a.create_partitions([])
172172

173173
with pytest.raises(Exception):
@@ -204,16 +204,16 @@ def test_describe_configs_api():
204204
fs = a.describe_configs([ConfigResource(confluent_kafka.RESOURCE_BROKER, "3")])
205205
# ignore the result
206206

207-
with pytest.raises(TypeError):
207+
with pytest.raises(Exception):
208208
a.describe_configs(None)
209209

210-
with pytest.raises(TypeError):
210+
with pytest.raises(Exception):
211211
a.describe_configs("something")
212212

213-
with pytest.raises(TypeError):
213+
with pytest.raises(Exception):
214214
a.describe_configs([])
215215

216-
with pytest.raises(TypeError):
216+
with pytest.raises(ValueError):
217217
a.describe_configs([None, ConfigResource(confluent_kafka.RESOURCE_TOPIC, "mytopic")])
218218

219219
fs = a.describe_configs([ConfigResource(confluent_kafka.RESOURCE_TOPIC, "mytopic"),
@@ -235,16 +235,16 @@ def test_alter_configs_api():
235235
set_config={"some": "config"})])
236236
# ignore the result
237237

238-
with pytest.raises(TypeError):
238+
with pytest.raises(Exception):
239239
a.alter_configs(None)
240240

241-
with pytest.raises(TypeError):
241+
with pytest.raises(Exception):
242242
a.alter_configs("something")
243243

244-
with pytest.raises(TypeError):
244+
with pytest.raises(ValueError):
245245
a.alter_configs([])
246246

247-
with pytest.raises(TypeError):
247+
with pytest.raises(ValueError):
248248
a.alter_configs([None, ConfigResource("topic", "mytopic",
249249
add_config={"something": "else"})])
250250

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