Skip to content

Commit 395f939

Browse files
authored
CLIENTS-1765: Added support for consumer.memberid() confluentinc#1154 (confluentinc#1455)
Added support for consumer.memberid()
1 parent d16a199 commit 395f939

File tree

3 files changed

+94
-2
lines changed

3 files changed

+94
-2
lines changed

CHANGELOG.md

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,9 +1,11 @@
11
# Confluent's Python client for Apache Kafka
22

33

4-
## v1.10.0
4+
## v1.9.3
55

6-
- Add metadata to TopicPartition type and commit() (#1410).
6+
- Added metadata to `TopicPartition` type and `commit()` (#1410).
7+
- Added `consumer.memberid()` for getting member id assigned to
8+
the consumer in a consumer group (#1154).
79

810

911
## v1.9.2

src/confluent_kafka/src/Consumer.c

Lines changed: 40 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -980,6 +980,33 @@ static PyObject *Consumer_poll (Handle *self, PyObject *args,
980980
}
981981

982982

983+
static PyObject *Consumer_memberid (Handle *self, PyObject *args,
984+
PyObject *kwargs) {
985+
char *memberid;
986+
PyObject *memberidobj;
987+
if (!self->rk) {
988+
PyErr_SetString(PyExc_RuntimeError,
989+
"Consumer closed");
990+
return NULL;
991+
}
992+
993+
memberid = rd_kafka_memberid(self->rk);
994+
995+
if (!memberid)
996+
Py_RETURN_NONE;
997+
998+
if (!*memberid) {
999+
rd_kafka_mem_free(self->rk, memberid);
1000+
Py_RETURN_NONE;
1001+
}
1002+
1003+
memberidobj = Py_BuildValue("s", memberid);
1004+
rd_kafka_mem_free(self->rk, memberid);
1005+
1006+
return memberidobj;
1007+
}
1008+
1009+
9831010
static PyObject *Consumer_consume (Handle *self, PyObject *args,
9841011
PyObject *kwargs) {
9851012
unsigned int num_messages = 1;
@@ -1409,6 +1436,19 @@ static PyMethodDef Consumer_methods[] = {
14091436
" :raises: RuntimeError if called on a closed consumer\n"
14101437
"\n"
14111438
},
1439+
{ "memberid", (PyCFunction)Consumer_memberid, METH_NOARGS,
1440+
".. py:function:: memberid()\n"
1441+
"\n"
1442+
" Return this client's broker-assigned group member id.\n"
1443+
"\n"
1444+
" The member id is assigned by the group coordinator and"
1445+
" is propagated to the consumer during rebalance.\n"
1446+
"\n"
1447+
" :returns: Member id string or None\n"
1448+
" :rtype: string\n"
1449+
" :raises: RuntimeError if called on a closed consumer\n"
1450+
"\n"
1451+
},
14121452
{ "close", (PyCFunction)Consumer_close, METH_NOARGS,
14131453
"\n"
14141454
" Close down and terminate the Kafka Consumer.\n"
Lines changed: 50 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,50 @@
1+
#!/usr/bin/env python
2+
# -*- coding: utf-8 -*-
3+
#
4+
# Copyright 2022 Confluent Inc.
5+
#
6+
# Licensed under the Apache License, Version 2.0 (the "License");
7+
# you may not use this file except in compliance with the License.
8+
# You may obtain a copy of the License at
9+
#
10+
# http://www.apache.org/licenses/LICENSE-2.0
11+
#
12+
# Unless required by applicable law or agreed to in writing, software
13+
# distributed under the License is distributed on an "AS IS" BASIS,
14+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
# See the License for the specific language governing permissions and
16+
# limit
17+
18+
import pytest
19+
20+
21+
def test_consumer_memberid(kafka_cluster):
22+
"""
23+
Test consumer memberid.
24+
"""
25+
26+
consumer_conf = {'group.id': 'test'}
27+
28+
topic = "testmemberid"
29+
30+
kafka_cluster.create_topic(topic)
31+
32+
consumer = kafka_cluster.consumer(consumer_conf)
33+
34+
assert consumer is not None
35+
assert consumer.memberid() is None
36+
kafka_cluster.seed_topic(topic, value_source=[b'memberid'])
37+
38+
consumer.subscribe([topic])
39+
msg = consumer.poll(10)
40+
assert msg is not None
41+
assert msg.value() == b'memberid'
42+
memberid = consumer.memberid()
43+
print("Member Id is -----> " + memberid)
44+
assert isinstance(memberid, str)
45+
assert len(memberid) > 0
46+
consumer.close()
47+
48+
with pytest.raises(RuntimeError) as error_info:
49+
consumer.memberid()
50+
assert error_info.value.args[0] == "Consumer closed"

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