-
Notifications
You must be signed in to change notification settings - Fork 934
expose stats_cb #55
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
expose stats_cb #55
Changes from 2 commits
201b6b3
8f88eb1
73b363f
d7ab475
eb31ace
90d11df
495a836
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -749,7 +749,7 @@ PyObject *c_parts_to_py (const rd_kafka_topic_partition_list_t *c_parts) { | |
|
|
||
| parts = PyList_New(c_parts->cnt); | ||
|
|
||
| for (i = 0 ; i < c_parts->cnt ; i++) { | ||
| for (i = 0 ; i < (size_t)c_parts->cnt ; i++) { | ||
| const rd_kafka_topic_partition_t *rktpar = &c_parts->elems[i]; | ||
| PyList_SET_ITEM(parts, i, | ||
| TopicPartition_new0( | ||
|
|
@@ -778,7 +778,7 @@ rd_kafka_topic_partition_list_t *py_to_c_parts (PyObject *plist) { | |
|
|
||
| c_parts = rd_kafka_topic_partition_list_new((int)PyList_Size(plist)); | ||
|
|
||
| for (i = 0 ; i < PyList_Size(plist) ; i++) { | ||
| for (i = 0 ; i < (size_t)PyList_Size(plist) ; i++) { | ||
| TopicPartition *tp = (TopicPartition *) | ||
| PyList_GetItem(plist, i); | ||
|
|
||
|
|
@@ -825,9 +825,9 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque) | |
| result = PyObject_CallFunctionObjArgs(h->error_cb, eo, NULL); | ||
| Py_DECREF(eo); | ||
|
|
||
| if (result) { | ||
| if (result) | ||
| Py_DECREF(result); | ||
| } else { | ||
| else { | ||
| CallState_crash(cs); | ||
| rd_kafka_yield(h->rk); | ||
| } | ||
|
|
@@ -836,25 +836,24 @@ static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque) | |
| CallState_resume(cs); | ||
| } | ||
|
|
||
| static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) | ||
| { | ||
| static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) { | ||
| Handle *h = opaque; | ||
| PyObject *eo=NULL, *result=NULL; | ||
| CallState *cs=NULL; | ||
| PyObject *eo = NULL, *result = NULL; | ||
| CallState *cs = NULL; | ||
|
|
||
| cs = CallState_get(h); | ||
| if (json_len== 0 || !h->stats_cb) { | ||
| /* Neither data nor call back defined. */ | ||
| if (json_len == 0) { | ||
| /* No data returned*/ | ||
| goto done; | ||
| } | ||
|
|
||
| eo = Py_BuildValue("s", json); | ||
| result = PyObject_CallFunctionObjArgs(h->stats_cb, eo, NULL); | ||
| Py_DECREF(eo); | ||
|
|
||
| if (result) { | ||
| if (result) | ||
| Py_DECREF(result); | ||
| } else { | ||
| else { | ||
| CallState_crash(cs); | ||
| rd_kafka_yield(h->rk); | ||
| } | ||
|
|
@@ -880,13 +879,11 @@ static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) | |
| * Clear Python object references in Handle | ||
| */ | ||
| void Handle_clear (Handle *h) { | ||
| if (h->error_cb) { | ||
| if (h->error_cb) | ||
|
||
| Py_DECREF(h->error_cb); | ||
| } | ||
|
|
||
| if (h->stats_cb) { | ||
| if (h->stats_cb) | ||
| Py_DECREF(h->stats_cb); | ||
| } | ||
|
|
||
| PyThread_delete_key(h->tlskey); | ||
| } | ||
|
|
@@ -1146,7 +1143,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype, | |
| Py_DECREF(ks); | ||
| continue; | ||
|
|
||
| } else if (!strcmp(k, "error_cb")) { | ||
| } else if (!strcmp(k, "error_cb") && PyCallable_Check(vo)) { | ||
|
||
| if (h->error_cb) { | ||
| Py_DECREF(h->error_cb); | ||
| h->error_cb = NULL; | ||
|
|
@@ -1157,7 +1154,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype, | |
| } | ||
| Py_DECREF(ks); | ||
| continue; | ||
| } else if (!strcmp(k, "stats_cb")) { | ||
| } else if (!strcmp(k, "stats_cb") && PyCallable_Check(vo)) { | ||
| if (h->stats_cb) { | ||
| Py_DECREF(h->stats_cb); | ||
| h->stats_cb = NULL; | ||
|
|
@@ -1220,9 +1217,8 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype, | |
| if (h->error_cb) | ||
| rd_kafka_conf_set_error_cb(conf, error_cb); | ||
|
|
||
| if (h->stats_cb) { | ||
| if (h->stats_cb) | ||
| rd_kafka_conf_set_stats_cb(conf, stats_cb); | ||
| } | ||
|
|
||
| rd_kafka_topic_conf_set_opaque(tconf, h); | ||
| rd_kafka_conf_set_default_topic_conf(conf, tconf); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import time | ||
| import uuid | ||
| import sys | ||
| import json | ||
|
|
||
| try: | ||
| from progress.bar import Bar | ||
|
|
@@ -39,6 +40,7 @@ | |
|
|
||
| def error_cb (err): | ||
| print('Error: %s' % err) | ||
|
|
||
|
|
||
| class MyTestDr(object): | ||
| """ Producer: Delivery report callback """ | ||
|
|
@@ -353,6 +355,78 @@ def my_on_revoke (consumer, partitions): | |
| c.close() | ||
|
|
||
|
|
||
| def verify_stats_cb(): | ||
| """ Verify stats_cb """ | ||
|
|
||
| def stats_cb(stats_json_str): | ||
| stats_json = json.loads(stats_json_str) | ||
| if 'test' in stats_json['topics']: | ||
| print("# app_offset stats for topic test partition 0: %d" % stats_json['topics']['test']['partitions']['0']['app_offset']) | ||
|
||
|
|
||
| conf = {'bootstrap.servers': bootstrap_servers, | ||
| 'group.id': uuid.uuid1(), | ||
| 'session.timeout.ms': 6000, | ||
| 'error_cb': error_cb, | ||
| 'stats_cb': stats_cb, | ||
| 'statistics.interval.ms': 200, | ||
| 'default.topic.config': { | ||
| 'auto.offset.reset': 'earliest' | ||
| }} | ||
|
|
||
| c = confluent_kafka.Consumer(**conf) | ||
| c.subscribe(["test"]) | ||
|
|
||
| max_msgcnt = 1000000 | ||
| bytecnt = 0 | ||
| msgcnt = 0 | ||
|
|
||
| print('Will now consume %d messages' % max_msgcnt) | ||
|
|
||
| if with_progress: | ||
| bar = Bar('Consuming', max=max_msgcnt, | ||
| suffix='%(index)d/%(max)d [%(eta_td)s]') | ||
| else: | ||
| bar = None | ||
|
|
||
| while True: | ||
|
||
| # Consume until EOF or error | ||
|
|
||
| msg = c.poll(timeout=20.0) | ||
| if msg is None: | ||
| raise Exception('Stalled at %d/%d message, no new messages for 20s' % | ||
| (msgcnt, max_msgcnt)) | ||
|
|
||
| if msg.error(): | ||
| if msg.error().code() == confluent_kafka.KafkaError._PARTITION_EOF: | ||
| # Reached EOF for a partition, ignore. | ||
| continue | ||
| else: | ||
| raise confluent_kafka.KafkaException(msg.error()) | ||
|
|
||
|
|
||
| bytecnt += len(msg) | ||
| msgcnt += 1 | ||
|
|
||
| if bar is not None and (msgcnt % 10000) == 0: | ||
| bar.next(n=10000) | ||
|
|
||
| if msgcnt == 1: | ||
| t_first_msg = time.time() | ||
| if msgcnt >= max_msgcnt: | ||
| break | ||
|
|
||
| if bar is not None: | ||
| bar.finish() | ||
|
|
||
| if msgcnt > 0: | ||
| t_spent = time.time() - t_first_msg | ||
| print('%d messages (%.2fMb) consumed in %.3fs: %d msgs/s, %.2f Mb/s' % \ | ||
| (msgcnt, bytecnt / (1024*1024), t_spent, msgcnt / t_spent, | ||
| (bytecnt / t_spent) / (1024*1024))) | ||
|
|
||
| print('closing consumer') | ||
| c.close() | ||
|
|
||
|
|
||
| if __name__ == '__main__': | ||
|
|
||
|
|
@@ -377,6 +451,9 @@ def my_on_revoke (consumer, partitions): | |
| print('=' * 30, 'Verifying Consumer performance', '=' * 30) | ||
| verify_consumer_performance() | ||
|
|
||
| print('=' * 30, 'Verifying stats_cb', '=' * 30) | ||
| verify_stats_cb() | ||
|
|
||
| print('=' * 30, 'Done', '=' * 30) | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,7 +1,9 @@ | ||
| #!/usr/bin/env python | ||
|
|
||
| import confluent_kafka | ||
|
|
||
| import json | ||
| import time | ||
| from pprint import pprint | ||
|
|
||
| def test_version(): | ||
| print('Using confluent_kafka module version %s (0x%x)' % confluent_kafka.version()) | ||
|
|
@@ -14,3 +16,54 @@ def test_version(): | |
| assert len(sver) > 0 | ||
| assert iver > 0 | ||
|
|
||
| def test_error_cb(): | ||
| """ Tests error_cb. """ | ||
|
|
||
| def error_cb(error_msg): | ||
| print('OK: error_cb() called') | ||
| assert error_msg.code() in (confluent_kafka.KafkaError._TRANSPORT, confluent_kafka.KafkaError._ALL_BROKERS_DOWN) | ||
|
|
||
| conf = {'bootstrap.servers': 'localhost:9093', # Purposely cause connection refused error | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is actually probably the second most likely port to find a broker on localhost, so maybe you want to connect to localhost:22 or somesuch that will definately not a Kafka broker. |
||
| 'group.id':'test', | ||
| 'socket.timeout.ms':'100', | ||
| 'session.timeout.ms': 1000, # Avoid close() blocking too long | ||
| 'error_cb': error_cb | ||
| } | ||
|
|
||
| kc = confluent_kafka.Consumer(**conf) | ||
|
|
||
| kc.subscribe(["test"]) | ||
| kc.poll(timeout=0.001) | ||
|
||
| time.sleep(1) | ||
| kc.unsubscribe() | ||
|
|
||
| kc.close() | ||
|
|
||
| def test_stats_cb(): | ||
| """ Tests stats_cb. """ | ||
|
|
||
| def stats_cb(stats_json_str): | ||
| # print(stats_json_str) | ||
| try: | ||
| stats_json = json.loads(stats_json_str) | ||
| if 'type' in stats_json: | ||
|
||
| print("stats_cb: type=%s" % stats_json['type']) | ||
| print('OK: stats_cb() called') | ||
| except Exception as e: | ||
| assert False | ||
|
|
||
| conf = {'group.id':'test', | ||
| 'socket.timeout.ms':'100', | ||
| 'session.timeout.ms': 1000, # Avoid close() blocking too long | ||
| 'statistics.interval.ms': 200, | ||
| 'stats_cb': stats_cb | ||
| } | ||
|
|
||
| kc = confluent_kafka.Consumer(**conf) | ||
|
|
||
| kc.subscribe(["test"]) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same thing as suggested for error_cb |
||
| kc.poll(timeout=0.001) | ||
| time.sleep(1) | ||
|
|
||
| kc.close() | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Make these size_t changes a separate commit, for tracking.