Skip to content

Commit 99b774b

Browse files
committed
on_commit(): removed consumer arg
1 parent 0dad9ef commit 99b774b

File tree

4 files changed

+5
-5
lines changed

4 files changed

+5
-5
lines changed

confluent_kafka/cimpl/Consumer.c

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -552,7 +552,7 @@ PyTypeObject ConsumerType = {
552552
"request has succeeded or failed.\n"
553553
"\n"
554554
"\n"
555-
".. py:function:: on_commit(consumer, err, partitions)\n"
555+
".. py:function:: on_commit(err, partitions)\n"
556556
"\n"
557557
" :param Consumer consumer: Consumer instance.\n"
558558
" :param KafkaError err: Commit error object, or None on success.\n"
@@ -656,7 +656,7 @@ static void Consumer_offset_commit_cb (rd_kafka_t *rk, rd_kafka_resp_err_t err,
656656
/* Construct list of TopicPartition based on 'c_parts' */
657657
parts = c_parts_to_py(c_parts);
658658

659-
args = Py_BuildValue("(OOO)", self, k_err, parts);
659+
args = Py_BuildValue("(OO)", k_err, parts);
660660

661661
Py_DECREF(k_err);
662662
Py_DECREF(parts);

confluent_kafka/kafkatest/kafkatest_verifiable_consumer.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,7 @@ def on_revoke (self, consumer, partitions):
105105
self.do_commit(immediate=True)
106106

107107

108-
def on_commit (self, consumer, err, partitions):
108+
def on_commit (self, err, partitions):
109109
""" Offsets Committed callback """
110110
if err is not None and err.code() == KafkaError._NO_OFFSET:
111111
self.dbg('on_commit(): no offsets to commit')

examples/integration_test.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -191,7 +191,7 @@ def verify_producer_performance(with_dr_cb=True):
191191
(t_delivery_spent - t_produce_spent))
192192

193193

194-
def print_commit_result (consumer, err, partitions):
194+
def print_commit_result (err, partitions):
195195
if err is not None:
196196
print('# Failed to commit offsets: %s: %s' % (err, partitions))
197197
else:

tests/test_Consumer.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,7 @@ def test_basic_api():
1212
except TypeError as e:
1313
assert str(e) == "expected configuration dict"
1414

15-
def dummy_commit_cb (consumer, err, partitions):
15+
def dummy_commit_cb (err, partitions):
1616
pass
1717

1818
kc = Consumer({'group.id':'test', 'socket.timeout.ms':'100',

0 commit comments

Comments
 (0)