Skip to content

Commit 979343a

Browse files
authored
[KIP-460] ElectLeaders implementation public API changes and documentation improvement (#1833)
- ElectionType confluent_kafka as in Java - dictionary value is a KafkaError as it's not thrown - ElectionType should be added to documentation .rst - docstring changes - tests without brokers with successful validation that leads to a timeout - example elects leaders for all partitions in case no partition is specified
1 parent 7085708 commit 979343a

File tree

10 files changed

+56
-74
lines changed

10 files changed

+56
-74
lines changed

docs/index.rst

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ Supporting classes
3232
- :ref:`ConsumerGroupTopicPartitions <pythonclient_consumer_group_topic_partition>`
3333
- :ref:`ConsumerGroupState <pythonclient_consumer_group_state>`
3434
- :ref:`Uuid <pythonclient_uuid>`
35+
- :ref:`ElectionType <pythonclient_election_type>`
3536

3637
- Errors:
3738
- :ref:`KafkaError <pythonclient_kafkaerror>`
@@ -701,6 +702,15 @@ Uuid
701702
.. autoclass:: confluent_kafka.Uuid
702703
:members:
703704

705+
.. _pythonclient_election_type:
706+
707+
************
708+
ElectionType
709+
************
710+
711+
.. autoclass:: confluent_kafka.ElectionType
712+
:members:
713+
704714
.. _serde_field:
705715

706716
************

examples/adminapi.py

Lines changed: 12 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -19,14 +19,14 @@
1919

2020
from confluent_kafka import (KafkaException, ConsumerGroupTopicPartitions,
2121
TopicPartition, ConsumerGroupState, TopicCollection,
22-
IsolationLevel)
22+
IsolationLevel, ElectionType)
2323
from confluent_kafka.admin import (AdminClient, NewTopic, NewPartitions, ConfigResource,
2424
ConfigEntry, ConfigSource, AclBinding,
2525
AclBindingFilter, ResourceType, ResourcePatternType,
2626
AclOperation, AclPermissionType, AlterConfigOpType,
2727
ScramMechanism, ScramCredentialInfo,
2828
UserScramCredentialUpsertion, UserScramCredentialDeletion,
29-
OffsetSpec, ElectionType)
29+
OffsetSpec)
3030
import sys
3131
import threading
3232
import logging
@@ -893,17 +893,24 @@ def example_elect_leaders(a, args):
893893
for topic, partition in zip(args[1::2], args[2::2]):
894894
partitions.append(TopicPartition(topic, int(partition)))
895895

896+
if len(partitions) == 0:
897+
# When passing None as partitions, election is triggered for
898+
# all partitions in the cluster
899+
partitions = None
900+
896901
f = a.elect_leaders(election_type, partitions)
897902
try:
898903
results = f.result()
899-
for partition, exception in results.items():
900-
if exception is None:
904+
print(f"Elect leaders call returned {len(results)} result(s):")
905+
for partition, error in results.items():
906+
if error is None:
901907
print(f"Leader Election Successful for topic: '{partition.topic}'" +
902908
f" partition: '{partition.partition}'")
903909
else:
904910
print(
905911
"Leader Election Failed for topic: " +
906-
f"'{partition.topic}' partition: '{partition.partition}': {exception}")
912+
f"'{partition.topic}' partition: '{partition.partition}' " +
913+
f"error code: {error.code()} error message: {error.str()}")
907914
except KafkaException as e:
908915
print(f"Error electing leaders: {e}")
909916

src/confluent_kafka/__init__.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,8 @@
2424
ConsumerGroupState,
2525
TopicCollection,
2626
TopicPartitionInfo,
27-
IsolationLevel)
27+
IsolationLevel,
28+
ElectionType)
2829

2930
from .cimpl import (Producer,
3031
Consumer,

src/confluent_kafka/_model/__init__.py

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -149,3 +149,22 @@ def __lt__(self, other):
149149
if self.__class__ != other.__class__:
150150
return NotImplemented
151151
return self.value < other.value
152+
153+
154+
class ElectionType(Enum):
155+
"""
156+
Enumerates the different types of leader elections.
157+
158+
Values:
159+
-------
160+
"""
161+
162+
#: Preferred election
163+
PREFERRED = cimpl.ELECTION_TYPE_PREFERRED
164+
#: Unclean election
165+
UNCLEAN = cimpl.ELECTION_TYPE_UNCLEAN
166+
167+
def __lt__(self, other):
168+
if self.__class__ != other.__class__:
169+
return NotImplemented
170+
return self.value < other.value

src/confluent_kafka/admin/__init__.py

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -54,11 +54,10 @@
5454
from ._listoffsets import (OffsetSpec, # noqa: F401
5555
ListOffsetsResultInfo)
5656

57-
from ._election import (ElectionType) # noqa: F401
58-
5957
from ._records import DeletedRecords # noqa: F401
6058

61-
from .._model import TopicCollection as _TopicCollection
59+
from .._model import (TopicCollection as _TopicCollection,
60+
ElectionType as _ElectionType)
6261

6362
from ..cimpl import (KafkaException, # noqa: F401
6463
KafkaError,
@@ -552,7 +551,7 @@ def _check_delete_records(request):
552551

553552
@staticmethod
554553
def _check_elect_leaders(election_type, partitions):
555-
if not isinstance(election_type, ElectionType):
554+
if not isinstance(election_type, _ElectionType):
556555
raise TypeError("Expected 'election_type' to be of type 'ElectionType'")
557556
if partitions is not None:
558557
if not isinstance(partitions, list):
@@ -1280,7 +1279,7 @@ def delete_records(self, topic_partition_offsets, **kwargs):
12801279
def elect_leaders(self, election_type, partitions=None, **kwargs):
12811280
"""
12821281
Perform Preferred or Unclean leader election for
1283-
all the specified topic partitions.
1282+
all the specified partitions or all partitions in the cluster.
12841283
12851284
:param ElectionType election_type: The type of election to perform.
12861285
:param List[TopicPartition]|None partitions: The topic partitions to perform

src/confluent_kafka/admin/_election.py

Lines changed: 0 additions & 29 deletions
This file was deleted.

src/confluent_kafka/src/Admin.c

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3152,7 +3152,7 @@ const char Admin_elect_leaders_doc[] = PyDoc_STR(
31523152
"future, [request_timeout, operation_timeout])\n"
31533153
"\n"
31543154
" Perform Preferred or Unclean election for the specified "
3155-
"Topic Partitions.\n"
3155+
"partion or all partition in the cluster.\n"
31563156
"\n"
31573157
" This method should not be used directly, use "
31583158
"confluent_kafka.AdminClient.elect_leaders()\n");

src/confluent_kafka/src/confluent_kafka.c

Lines changed: 2 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -408,33 +408,6 @@ static void cfl_PyErr_Fatal (rd_kafka_resp_err_t err, const char *reason) {
408408
PyErr_SetObject(KafkaException, eo);
409409
}
410410

411-
/**
412-
* @brief Creates a KafkaException from error code and error string.
413-
*/
414-
PyObject *KafkaException_new_or_none (rd_kafka_resp_err_t err, const char *str) {
415-
if (err) {
416-
PyObject *excargs , *exc;
417-
PyObject *error = KafkaError_new0(err, str);
418-
419-
excargs = PyTuple_New(1);
420-
PyTuple_SetItem(excargs, 0, error);
421-
422-
exc = ((PyTypeObject *)KafkaException)->tp_new(
423-
(PyTypeObject *)KafkaException, NULL, NULL);
424-
exc->ob_type->tp_init(exc, excargs, NULL);
425-
426-
Py_DECREF(excargs);
427-
Py_DECREF(error);
428-
429-
return exc;
430-
}
431-
else
432-
Py_RETURN_NONE;
433-
}
434-
435-
436-
437-
438411

439412
/****************************************************************************
440413
*
@@ -1403,8 +1376,8 @@ PyObject *c_topic_partition_result_to_py_dict(
14031376
rd_kafka_topic_partition_result_partition(partition_results[i]);
14041377
c_error = rd_kafka_topic_partition_result_error(partition_results[i]);
14051378

1406-
value = KafkaException_new_or_none(rd_kafka_error_code(c_error),
1407-
rd_kafka_error_string(c_error));
1379+
value = KafkaError_new_or_None(rd_kafka_error_code(c_error),
1380+
rd_kafka_error_string(c_error));
14081381
key = c_part_to_py(c_topic_partition);
14091382

14101383
PyDict_SetItem(result, key, value);

src/confluent_kafka/src/confluent_kafka.h

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -188,7 +188,6 @@ extern PyObject *KafkaException;
188188
PyObject *KafkaError_new0 (rd_kafka_resp_err_t err, const char *fmt, ...);
189189
PyObject *KafkaError_new_or_None (rd_kafka_resp_err_t err, const char *str);
190190
PyObject *KafkaError_new_from_error_destroy (rd_kafka_error_t *error);
191-
PyObject *KafkaException_new_or_none (rd_kafka_resp_err_t err, const char *str);
192191

193192
/**
194193
* @brief Raise an exception using KafkaError.

tests/test_Admin.py

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -6,11 +6,10 @@
66
ResourcePatternType, AclOperation, AclPermissionType, AlterConfigOpType, \
77
ScramCredentialInfo, ScramMechanism, \
88
UserScramCredentialAlteration, UserScramCredentialDeletion, \
9-
UserScramCredentialUpsertion, OffsetSpec, \
10-
ElectionType
9+
UserScramCredentialUpsertion, OffsetSpec
1110
from confluent_kafka import KafkaException, KafkaError, libversion, \
1211
TopicPartition, ConsumerGroupTopicPartitions, ConsumerGroupState, \
13-
IsolationLevel, TopicCollection
12+
IsolationLevel, TopicCollection, ElectionType
1413
import concurrent.futures
1514

1615

@@ -1230,3 +1229,7 @@ def test_elect_leaders():
12301229

12311230
with pytest.raises(ValueError):
12321231
a.elect_leaders(correct_election_type, [incorrect_partitions])
1232+
1233+
with pytest.raises(KafkaException):
1234+
a.elect_leaders(correct_election_type, [correct_partitions])\
1235+
.result(timeout=1)

0 commit comments

Comments
 (0)