Skip to content

Commit 1db3054

Browse files
pranavrthemasab
andauthored
Implemented KIP-88, KIP-222, KIP-229, KIP-518 and partially KIP-396 (confluentinc#1449)
Implement KIP-88, KIP-222, KIP-229, KIP-518 and partially KIP-396 (#3995) Add support for: KIP-88, KIP-222: ListConsumerGroups, DescribeConsumerGroups, ListConsumerGroupOffsets, KIP-229: DeleteConsumerGroups, KIP-396: AlterConsumerGroupOffsets, KIP-518. Co-authored-by: Emanuele Sabellico <esabellico@confluent.io>
1 parent 38d0f03 commit 1db3054

27 files changed

+2711
-343
lines changed

.semaphore/semaphore.yml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,7 @@ agent:
66
global_job_config:
77
env_vars:
88
- name: LIBRDKAFKA_VERSION
9-
value: v1.9.2
9+
value: v2.0.0-RC3
1010
prologue:
1111
commands:
1212
- export HOME=$WORKSPACE

.travis.yml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
env:
22
global:
3-
- LIBRDKAFKA_VERSION=v1.9.2
3+
- LIBRDKAFKA_VERSION=v2.0.0-RC3
44
- LIBRDKAFKA_SRC_VERSION=master
55

66
jobs:

CHANGELOG.md

Lines changed: 22 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -4,9 +4,29 @@
44
## Next Version
55

66
- Added metadata to `TopicPartition` type and `commit()` (#1410).
7-
- Added `consumer.memberid()` for getting member id assigned to
7+
- Added `consumer.memberid()` for getting member id assigned to
88
the consumer in a consumer group (#1154).
9-
- Added Python 3.11 wheels
9+
- Implemented `nb_bool` method for the Producer, so that the default (which uses len)
10+
will not be used. This avoids situations where producers with no enqueued items would
11+
evaluate to False (#1445).
12+
- Added Python 3.11 wheels.
13+
- [KIP-222](https://cwiki.apache.org/confluence/display/KAFKA/KIP-222+-+Add+Consumer+Group+operations+to+Admin+API)
14+
Add Consumer Group operations to Admin API.
15+
- [KIP-518](https://cwiki.apache.org/confluence/display/KAFKA/KIP-518%3A+Allow+listing+consumer+groups+per+state)
16+
Allow listing consumer groups per state.
17+
- [KIP-396](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=97551484)
18+
Partially implemented: support for AlterConsumerGroupOffsets.
19+
- As result of the above KIPs, added (#1449)
20+
- `list_consumer_groups` Admin operation. Supports listing by state.
21+
- `describe_consumer_groups` Admin operation. Supports multiple groups.
22+
- `delete_consumer_groups` Admin operation. Supports multiple groups.
23+
- `list_consumer_group_offsets` Admin operation. Currently, only supports 1 group with multiple partitions. Supports require_stable option.
24+
- `alter_consumer_group_offsets` Admin operation. Currently, only supports 1 group with multiple offsets.
25+
- Added `normalize.schemas` configuration property to Schema Registry client
26+
27+
confluent-kafka-python is based on librdkafka v2.0.0, see the
28+
[librdkafka release notes](https://github.com/edenhill/librdkafka/releases/tag/v2.0.0)
29+
for a complete list of changes, enhancements, fixes and upgrade considerations.
1030

1131

1232
## v1.9.2

examples/adminapi.py

Lines changed: 149 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,11 @@
1717

1818
# Example use of AdminClient operations.
1919

20+
from confluent_kafka import (KafkaException, ConsumerGroupTopicPartitions,
21+
TopicPartition, ConsumerGroupState)
2022
from confluent_kafka.admin import (AdminClient, NewTopic, NewPartitions, ConfigResource, ConfigSource,
21-
AclBinding, AclBindingFilter, ResourceType, ResourcePatternType,
22-
AclOperation, AclPermissionType)
23-
from confluent_kafka import KafkaException
23+
AclBinding, AclBindingFilter, ResourceType, ResourcePatternType, AclOperation,
24+
AclPermissionType)
2425
import sys
2526
import threading
2627
import logging
@@ -419,17 +420,146 @@ def example_list(a, args):
419420
print(" {} consumer groups".format(len(groups)))
420421
for g in groups:
421422
if g.error is not None:
422-
errstr = ": {}".format(t.error)
423+
errstr = ": {}".format(g.error)
423424
else:
424425
errstr = ""
425426

426427
print(" \"{}\" with {} member(s), protocol: {}, protocol_type: {}{}".format(
427-
g, len(g.members), g.protocol, g.protocol_type, errstr))
428+
g, len(g.members), g.protocol, g.protocol_type, errstr))
428429

429430
for m in g.members:
430431
print("id {} client_id: {} client_host: {}".format(m.id, m.client_id, m.client_host))
431432

432433

434+
def example_list_consumer_groups(a, args):
435+
"""
436+
List Consumer Groups
437+
"""
438+
states = {ConsumerGroupState[state] for state in args}
439+
future = a.list_consumer_groups(request_timeout=10, states=states)
440+
try:
441+
list_consumer_groups_result = future.result()
442+
print("{} consumer groups".format(len(list_consumer_groups_result.valid)))
443+
for valid in list_consumer_groups_result.valid:
444+
print(" id: {} is_simple: {} state: {}".format(
445+
valid.group_id, valid.is_simple_consumer_group, valid.state))
446+
print("{} errors".format(len(list_consumer_groups_result.errors)))
447+
for error in list_consumer_groups_result.errors:
448+
print(" error: {}".format(error))
449+
except Exception:
450+
raise
451+
452+
453+
def example_describe_consumer_groups(a, args):
454+
"""
455+
Describe Consumer Groups
456+
"""
457+
458+
futureMap = a.describe_consumer_groups(args, request_timeout=10)
459+
460+
for group_id, future in futureMap.items():
461+
try:
462+
g = future.result()
463+
print("Group Id: {}".format(g.group_id))
464+
print(" Is Simple : {}".format(g.is_simple_consumer_group))
465+
print(" State : {}".format(g.state))
466+
print(" Partition Assignor : {}".format(g.partition_assignor))
467+
print(" Coordinator : ({}) {}:{}".format(g.coordinator.id, g.coordinator.host, g.coordinator.port))
468+
print(" Members: ")
469+
for member in g.members:
470+
print(" Id : {}".format(member.member_id))
471+
print(" Host : {}".format(member.host))
472+
print(" Client Id : {}".format(member.client_id))
473+
print(" Group Instance Id : {}".format(member.group_instance_id))
474+
if member.assignment:
475+
print(" Assignments :")
476+
for toppar in member.assignment.topic_partitions:
477+
print(" {} [{}]".format(toppar.topic, toppar.partition))
478+
except KafkaException as e:
479+
print("Error while describing group id '{}': {}".format(group_id, e))
480+
except Exception:
481+
raise
482+
483+
484+
def example_delete_consumer_groups(a, args):
485+
"""
486+
Delete Consumer Groups
487+
"""
488+
groups = a.delete_consumer_groups(args, request_timeout=10)
489+
for group_id, future in groups.items():
490+
try:
491+
future.result() # The result itself is None
492+
print("Deleted group with id '" + group_id + "' successfully")
493+
except KafkaException as e:
494+
print("Error deleting group id '{}': {}".format(group_id, e))
495+
except Exception:
496+
raise
497+
498+
499+
def example_list_consumer_group_offsets(a, args):
500+
"""
501+
List consumer group offsets
502+
"""
503+
504+
topic_partitions = []
505+
for topic, partition in zip(args[1::2], args[2::2]):
506+
topic_partitions.append(TopicPartition(topic, int(partition)))
507+
if len(topic_partitions) == 0:
508+
topic_partitions = None
509+
groups = [ConsumerGroupTopicPartitions(args[0], topic_partitions)]
510+
511+
futureMap = a.list_consumer_group_offsets(groups)
512+
513+
for group_id, future in futureMap.items():
514+
try:
515+
response_offset_info = future.result()
516+
print("Group: " + response_offset_info.group_id)
517+
for topic_partition in response_offset_info.topic_partitions:
518+
if topic_partition.error:
519+
print(" Error: " + topic_partition.error.str() + " occurred with " +
520+
topic_partition.topic + " [" + str(topic_partition.partition) + "]")
521+
else:
522+
print(" " + topic_partition.topic +
523+
" [" + str(topic_partition.partition) + "]: " + str(topic_partition.offset))
524+
525+
except KafkaException as e:
526+
print("Failed to list {}: {}".format(group_id, e))
527+
except Exception:
528+
raise
529+
530+
531+
def example_alter_consumer_group_offsets(a, args):
532+
"""
533+
Alter consumer group offsets
534+
"""
535+
536+
topic_partitions = []
537+
for topic, partition, offset in zip(args[1::3], args[2::3], args[3::3]):
538+
topic_partitions.append(TopicPartition(topic, int(partition), int(offset)))
539+
if len(topic_partitions) == 0:
540+
topic_partitions = None
541+
groups = [ConsumerGroupTopicPartitions(args[0], topic_partitions)]
542+
543+
futureMap = a.alter_consumer_group_offsets(groups)
544+
545+
for group_id, future in futureMap.items():
546+
try:
547+
response_offset_info = future.result()
548+
print("Group: " + response_offset_info.group_id)
549+
for topic_partition in response_offset_info.topic_partitions:
550+
if topic_partition.error:
551+
print(" Error: " + topic_partition.error.str() + " occurred with " +
552+
topic_partition.topic + " [" + str(topic_partition.partition) + "]")
553+
else:
554+
print(" " + topic_partition.topic +
555+
" [" + str(topic_partition.partition) + "]: " + str(topic_partition.offset))
556+
557+
except KafkaException as e:
558+
print("Failed to alter {}: {}".format(group_id, e))
559+
except Exception:
560+
raise
561+
562+
433563
if __name__ == '__main__':
434564
if len(sys.argv) < 3:
435565
sys.stderr.write('Usage: %s <bootstrap-brokers> <operation> <args..>\n\n' % sys.argv[0])
@@ -449,6 +579,14 @@ def example_list(a, args):
449579
sys.stderr.write(' delete_acls <resource_type1> <resource_name1> <resource_patter_type1> ' +
450580
'<principal1> <host1> <operation1> <permission_type1> ..\n')
451581
sys.stderr.write(' list [<all|topics|brokers|groups>]\n')
582+
sys.stderr.write(' list_consumer_groups [<state1> <state2> ..]\n')
583+
sys.stderr.write(' describe_consumer_groups <group1> <group2> ..\n')
584+
sys.stderr.write(' delete_consumer_groups <group1> <group2> ..\n')
585+
sys.stderr.write(' list_consumer_group_offsets <group> [<topic1> <partition1> <topic2> <partition2> ..]\n')
586+
sys.stderr.write(
587+
' alter_consumer_group_offsets <group> <topic1> <partition1> <offset1> ' +
588+
'<topic2> <partition2> <offset2> ..\n')
589+
452590
sys.exit(1)
453591

454592
broker = sys.argv[1]
@@ -467,7 +605,12 @@ def example_list(a, args):
467605
'create_acls': example_create_acls,
468606
'describe_acls': example_describe_acls,
469607
'delete_acls': example_delete_acls,
470-
'list': example_list}
608+
'list': example_list,
609+
'list_consumer_groups': example_list_consumer_groups,
610+
'describe_consumer_groups': example_describe_consumer_groups,
611+
'delete_consumer_groups': example_delete_consumer_groups,
612+
'list_consumer_group_offsets': example_list_consumer_group_offsets,
613+
'alter_consumer_group_offsets': example_alter_consumer_group_offsets}
471614

472615
if operation not in opsmap:
473616
sys.stderr.write('Unknown operation: %s\n' % operation)

src/confluent_kafka/__init__.py

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
from .deserializing_consumer import DeserializingConsumer
2020
from .serializing_producer import SerializingProducer
2121
from .error import KafkaException, KafkaError
22+
from ._model import Node, ConsumerGroupTopicPartitions, ConsumerGroupState
2223

2324
from .cimpl import (Producer,
2425
Consumer,
@@ -40,7 +41,8 @@
4041
'OFFSET_BEGINNING', 'OFFSET_END', 'OFFSET_INVALID', 'OFFSET_STORED',
4142
'Producer', 'DeserializingConsumer',
4243
'SerializingProducer', 'TIMESTAMP_CREATE_TIME', 'TIMESTAMP_LOG_APPEND_TIME',
43-
'TIMESTAMP_NOT_AVAILABLE', 'TopicPartition']
44+
'TIMESTAMP_NOT_AVAILABLE', 'TopicPartition', 'Node',
45+
'ConsumerGroupTopicPartitions', 'ConsumerGroupState']
4446

4547
__version__ = version()[0]
4648

Lines changed: 86 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,86 @@
1+
# Copyright 2022 Confluent Inc.
2+
#
3+
# Licensed under the Apache License, Version 2.0 (the "License");
4+
# you may not use this file except in compliance with the License.
5+
# You may obtain a copy of the License at
6+
#
7+
# http://www.apache.org/licenses/LICENSE-2.0
8+
#
9+
# Unless required by applicable law or agreed to in writing, software
10+
# distributed under the License is distributed on an "AS IS" BASIS,
11+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
# See the License for the specific language governing permissions and
13+
# limitations under the License.
14+
15+
from enum import Enum
16+
from .. import cimpl
17+
18+
19+
class Node:
20+
"""
21+
Represents node information.
22+
Used by :class:`ConsumerGroupDescription`
23+
24+
Parameters
25+
----------
26+
id: int
27+
The node id of this node.
28+
id_string:
29+
String representation of the node id.
30+
host:
31+
The host name for this node.
32+
port: int
33+
The port for this node.
34+
rack: str
35+
The rack for this node.
36+
"""
37+
def __init__(self, id, host, port, rack=None):
38+
self.id = id
39+
self.id_string = str(id)
40+
self.host = host
41+
self.port = port
42+
self.rack = rack
43+
44+
45+
class ConsumerGroupTopicPartitions:
46+
"""
47+
Represents consumer group and its topic partition information.
48+
Used by :meth:`AdminClient.list_consumer_group_offsets` and
49+
:meth:`AdminClient.alter_consumer_group_offsets`.
50+
51+
Parameters
52+
----------
53+
group_id: str
54+
Id of the consumer group.
55+
topic_partitions: list(TopicPartition)
56+
List of topic partitions information.
57+
"""
58+
def __init__(self, group_id, topic_partitions=None):
59+
self.group_id = group_id
60+
self.topic_partitions = topic_partitions
61+
62+
63+
class ConsumerGroupState(Enum):
64+
"""
65+
Enumerates the different types of Consumer Group State.
66+
67+
Values
68+
------
69+
UNKOWN : State is not known or not set.
70+
PREPARING_REBALANCING : Preparing rebalance for the consumer group.
71+
COMPLETING_REBALANCING : Consumer Group is completing rebalancing.
72+
STABLE : Consumer Group is stable.
73+
DEAD : Consumer Group is Dead.
74+
EMPTY : Consumer Group is Empty.
75+
"""
76+
UNKOWN = cimpl.CONSUMER_GROUP_STATE_UNKNOWN
77+
PREPARING_REBALANCING = cimpl.CONSUMER_GROUP_STATE_PREPARING_REBALANCE
78+
COMPLETING_REBALANCING = cimpl.CONSUMER_GROUP_STATE_COMPLETING_REBALANCE
79+
STABLE = cimpl.CONSUMER_GROUP_STATE_STABLE
80+
DEAD = cimpl.CONSUMER_GROUP_STATE_DEAD
81+
EMPTY = cimpl.CONSUMER_GROUP_STATE_EMPTY
82+
83+
def __lt__(self, other):
84+
if self.__class__ != other.__class__:
85+
return NotImplemented
86+
return self.value < other.value

src/confluent_kafka/_util/__init__.py

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
# Copyright 2022 Confluent Inc.
2+
#
3+
# Licensed under the Apache License, Version 2.0 (the "License");
4+
# you may not use this file except in compliance with the License.
5+
# You may obtain a copy of the License at
6+
#
7+
# http://www.apache.org/licenses/LICENSE-2.0
8+
#
9+
# Unless required by applicable law or agreed to in writing, software
10+
# distributed under the License is distributed on an "AS IS" BASIS,
11+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
# See the License for the specific language governing permissions and
13+
# limitations under the License.
14+
15+
from .validation_util import ValidationUtil # noqa: F401
16+
from .conversion_util import ConversionUtil # noqa: F401

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