/** * Helper: wait for single op on 'rkq', and return its error, * or .._TIMED_OUT on timeout. */ rd_kafka_resp_err_t rd_kafka_q_wait_result (rd_kafka_q_t *rkq, int timeout_ms) { rd_kafka_op_t *rko; rd_kafka_resp_err_t err; rko = rd_kafka_q_pop(rkq, timeout_ms, 0); if (!rko) err = RD_KAFKA_RESP_ERR__TIMED_OUT; else { err = rko->rko_err; rd_kafka_op_destroy(rko); } return err; }
/** * Filters out outdated ops. */ static RD_INLINE rd_kafka_op_t *rd_kafka_op_filter (rd_kafka_q_t *rkq, rd_kafka_op_t *rko) { if (unlikely(!rko)) return NULL; if (unlikely(rko->rko_version && rko->rko_rktp && rko->rko_version < rd_atomic32_get(&rd_kafka_toppar_s2i(rko->rko_rktp)-> rktp_version))) { rd_kafka_q_deq0(rkq, rko); rd_kafka_op_destroy(rko); return NULL; } return rko; }
rd_kafka_resp_err_t rd_kafka_commit_queue (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, rd_kafka_queue_t *rkqu, void (*cb) (rd_kafka_t *rk, rd_kafka_resp_err_t err, rd_kafka_topic_partition_list_t *offsets, void *opaque), void *opaque) { rd_kafka_q_t *rkq; rd_kafka_resp_err_t err; if (!rd_kafka_cgrp_get(rk)) return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP; if (rkqu) rkq = rkqu->rkqu_q; else rkq = rd_kafka_q_new(rk); err = rd_kafka_commit0(rk, offsets, NULL, RD_KAFKA_REPLYQ(rkq, 0), cb, opaque); if (!rkqu) { rd_kafka_op_t *rko = rd_kafka_q_pop(rkq, RD_POLL_INFINITE, 0); if (!rko) err = RD_KAFKA_RESP_ERR__TIMED_OUT; else { err = rko->rko_err; rd_kafka_op_handle_std(rk, rko); rd_kafka_op_destroy(rko); } rd_kafka_q_destroy(rkq); } return err; }
/** * Purge all entries from a queue with a rktp version smaller than `version` * This shaves off the head of the queue, up until the first rko with * a non-matching rktp or version. */ void rd_kafka_q_purge_toppar_version (rd_kafka_q_t *rkq, rd_kafka_toppar_t *rktp, int version) { rd_kafka_op_t *rko, *next; TAILQ_HEAD(, rd_kafka_op_s) tmpq = TAILQ_HEAD_INITIALIZER(tmpq); int32_t cnt = 0; int64_t size = 0; mtx_lock(&rkq->rkq_lock); if (rkq->rkq_fwdq) { rd_kafka_q_purge_toppar_version(rkq->rkq_fwdq, rktp, version); mtx_unlock(&rkq->rkq_lock); return; } /* Move ops to temporary queue and then destroy them from there * without locks to avoid lock-ordering problems in op_destroy() */ while ((rko = TAILQ_FIRST(&rkq->rkq_q)) && rko->rko_rktp && rd_kafka_toppar_s2i(rko->rko_rktp) == rktp && rko->rko_version < version) { TAILQ_REMOVE(&rkq->rkq_q, rko, rko_link); TAILQ_INSERT_TAIL(&tmpq, rko, rko_link); cnt++; size += rko->rko_len; } rkq->rkq_qlen -= cnt; rkq->rkq_qsize -= size; mtx_unlock(&rkq->rkq_lock); next = TAILQ_FIRST(&tmpq); while ((rko = next)) { next = TAILQ_NEXT(next, rko_link); rd_kafka_op_destroy(rko); } }
/** * NOTE: 'offsets' may be NULL, see official documentation. */ rd_kafka_resp_err_t rd_kafka_commit (rd_kafka_t *rk, const rd_kafka_topic_partition_list_t *offsets, int async) { rd_kafka_cgrp_t *rkcg; rd_kafka_resp_err_t err; rd_kafka_q_t *tmpq = NULL; if (!(rkcg = rd_kafka_cgrp_get(rk))) return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP; if (!async) tmpq = rd_kafka_q_new(rk); err = rd_kafka_commit0(rk, offsets, async ? &rkcg->rkcg_ops : tmpq, NULL); if (!async) { rd_kafka_op_t *rko = rd_kafka_q_pop(tmpq, RD_POLL_INFINITE, 0); err = rko->rko_err; /* Enqueue offset_commit_cb if configured */ if (rko->rko_payload /* offset list */) { rd_kafka_offset_commit_cb_op( rk, rko->rko_err, (rd_kafka_topic_partition_list_t *) rko->rko_payload); rko->rko_payload = NULL; } rd_kafka_op_destroy(rko); rd_kafka_q_destroy(tmpq); } else { err = RD_KAFKA_RESP_ERR_NO_ERROR; } return err; }
rd_kafka_resp_err_t rd_kafka_metadata (rd_kafka_t *rk, int all_topics, rd_kafka_topic_t *only_rkt, const struct rd_kafka_metadata **metadatap, int timeout_ms) { rd_kafka_q_t *rkq; rd_kafka_broker_t *rkb; rd_kafka_op_t *rko; rd_ts_t ts_end = rd_timeout_init(timeout_ms); rd_list_t topics; /* Query any broker that is up, and if none are up pick the first one, * if we're lucky it will be up before the timeout */ rkb = rd_kafka_broker_any_usable(rk, timeout_ms, 1, "application metadata request"); if (!rkb) return RD_KAFKA_RESP_ERR__TRANSPORT; rkq = rd_kafka_q_new(rk); rd_list_init(&topics, 0, rd_free); if (!all_topics) { if (only_rkt) rd_list_add(&topics, rd_strdup(rd_kafka_topic_a2i(only_rkt)-> rkt_topic->str)); else rd_kafka_local_topics_to_list(rkb->rkb_rk, &topics); } /* Async: request metadata */ rko = rd_kafka_op_new(RD_KAFKA_OP_METADATA); rd_kafka_op_set_replyq(rko, rkq, 0); rko->rko_u.metadata.force = 1; /* Force metadata request regardless * of outstanding metadata requests. */ rd_kafka_MetadataRequest(rkb, &topics, "application requested", rko); rd_list_destroy(&topics); rd_kafka_broker_destroy(rkb); /* Wait for reply (or timeout) */ rko = rd_kafka_q_pop(rkq, rd_timeout_remains(ts_end), 0); rd_kafka_q_destroy_owner(rkq); /* Timeout */ if (!rko) return RD_KAFKA_RESP_ERR__TIMED_OUT; /* Error */ if (rko->rko_err) { rd_kafka_resp_err_t err = rko->rko_err; rd_kafka_op_destroy(rko); return err; } /* Reply: pass metadata pointer to application who now owns it*/ rd_kafka_assert(rk, rko->rko_u.metadata.md); *metadatap = rko->rko_u.metadata.md; rko->rko_u.metadata.md = NULL; rd_kafka_op_destroy(rko); return RD_KAFKA_RESP_ERR_NO_ERROR; }
int main (int argc, char **argv) { rd_kafka_t *rk; char *broker = NULL; char mode = 'C'; char *topic = NULL; int partition = 0; int opt; int msgsize = 1024; int msgcnt = -1; int sendflags = 0; int dispintvl = 1000; struct { rd_ts_t t_start; rd_ts_t t_end; rd_ts_t t_end_send; uint64_t msgs; uint64_t bytes; rd_ts_t t_latency; rd_ts_t t_last; rd_ts_t t_total; } cnt = {}; rd_ts_t now; char *dirstr = ""; while ((opt = getopt(argc, argv, "PCt:p:b:s:c:fi:D")) != -1) { switch (opt) { case 'P': case 'C': mode = opt; break; case 't': topic = optarg; break; case 'p': partition = atoi(optarg); break; case 'b': broker = optarg; break; case 's': msgsize = atoi(optarg); break; case 'c': msgcnt = atoi(optarg); break; case 'D': sendflags |= RD_KAFKA_OP_F_FREE; break; case 'i': dispintvl = atoi(optarg); break; default: goto usage; } } if (!topic || optind != argc) { usage: fprintf(stderr, "Usage: %s [-C|-P] -t <topic> " "[-p <partition>] [-b <broker>] [options..]\n" "\n" " Options:\n" " -C | -P Consumer or Producer mode\n" " -t <topic> Topic to fetch / produce\n" " -p <num> Partition (defaults to 0)\n" " -b <broker> Broker address (localhost:9092)\n" " -s <size> Message size (producer)\n" " -c <cnt> Messages to transmit/receive\n" " -D Copy/Duplicate data buffer (producer)\n" " -i <ms> Display interval\n" "\n" " In Consumer mode:\n" " consumes messages and prints thruput\n" " In Producer mode:\n" " writes messages of size -s <..> and prints thruput\n" "\n", argv[0]); exit(1); } dispintvl *= 1000; /* us */ signal(SIGINT, stop); /* Socket hangups are gracefully handled in librdkafka on socket error * without the use of signals, so SIGPIPE should be ignored by the * calling program. */ signal(SIGPIPE, SIG_IGN); if (mode == 'P') { /* * Producer */ char *sbuf = malloc(msgsize); int endwait; int outq; int i; memset(sbuf, 'R', msgsize); if (msgcnt == -1) printf("%% Sending messages of size %i bytes\n", msgsize); else printf("%% Sending %i messages of size %i bytes\n", msgcnt ,msgsize); /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_PRODUCER, broker, NULL))) { perror("kafka_new producer"); exit(1); } cnt.t_start = rd_clock(); while (run && (msgcnt == -1 || cnt.msgs < msgcnt)) { char *pbuf = sbuf; /* Send/Produce message. */ if (sendflags & RD_KAFKA_OP_F_FREE) { /* Duplicate memory */ pbuf = malloc(msgsize); memcpy(pbuf, sbuf, msgsize); } rd_kafka_produce(rk, topic, partition, sendflags, pbuf, msgsize); cnt.msgs++; cnt.bytes += msgsize; now = rd_clock(); if (cnt.t_last + dispintvl <= now) { printf("%% %"PRIu64" messages and %"PRIu64 "bytes: %"PRIu64" msgs/s and " "%.2f Mb/s\n", cnt.msgs, cnt.bytes, (cnt.msgs / (now - cnt.t_start)) * 1000000, (float)(cnt.bytes / (now - cnt.t_start))); cnt.t_last = now; } } /* Wait for messaging to finish. */ i = 0; while (run && rd_kafka_outq_len(rk) > 0) { if (!(i++ % (dispintvl/1000))) printf("%% Waiting for %i messages in outq " "to be sent. Abort with Ctrl-c\n", rd_kafka_outq_len(rk)); usleep(1000); } cnt.t_end_send = rd_clock(); outq = rd_kafka_outq_len(rk); cnt.msgs -= outq; cnt.bytes -= msgsize * outq; cnt.t_end = rd_clock(); /* Since there is no ack for produce messages in 0.7 * we wait some more for any packets in the socket buffers * to be sent. * This is fixed in protocol version 0.8 */ endwait = cnt.msgs * 10; printf("%% Test timers stopped, but waiting %ims more " "for the %"PRIu64 " messages to be transmitted from " "socket buffers.\n" "%% End with Ctrl-c\n", endwait / 1000, cnt.msgs); run = 1; while (run && endwait > 0) { usleep(10000); endwait -= 10000; } /* Destroy the handle */ rd_kafka_destroy(rk); dirstr = "sent"; } else if (mode == 'C') { /* * Consumer */ rd_kafka_op_t *rko; /* Base our configuration on the default config. */ rd_kafka_conf_t conf = rd_kafka_defaultconf; /* The offset storage file is optional but its presence * avoids starting all over from offset 0 again when * the program restarts. * ZooKeeper functionality will be implemented in future * versions and then the offset will be stored there instead. */ conf.consumer.offset_file = "."; /* current directory */ /* Indicate to rdkafka that the application is responsible * for storing the offset. This allows the application to * succesfully handle a message before storing the offset. * If this flag is not set rdkafka will store the offset * just prior to returning the message from rd_kafka_consume(). */ conf.flags |= RD_KAFKA_CONF_F_APP_OFFSET_STORE; /* Tell rdkafka to (try to) maintain 10000 messages * in its internal receive buffers. This is to avoid * application -> rdkafka -> broker per-message ping-pong * latency. */ conf.consumer.replyq_low_thres = 100000; /* Use the consumer convenience function * to create a Kafka handle. */ if (!(rk = rd_kafka_new_consumer(broker, topic, (uint32_t)partition, 0, &conf))) { perror("kafka_new_consumer"); exit(1); } cnt.t_start = rd_clock(); while (run && (msgcnt == -1 || msgcnt > cnt.msgs)) { /* Fetch an "op" which is one of: * - a kafka message (if rko_len>0 && rko_err==0) * - an error (if rko_err) */ uint64_t latency; latency = rd_clock(); if (!(rko = rd_kafka_consume(rk, 1000/*timeout ms*/))) continue; cnt.t_latency += rd_clock() - latency; if (rko->rko_err) fprintf(stderr, "%% Error: %.*s\n", rko->rko_len, rko->rko_payload); else if (rko->rko_len) { cnt.msgs++; cnt.bytes += rko->rko_len; } /* rko_offset contains the offset of the _next_ * message. We store it when we're done processing * the current message. */ if (rko->rko_offset) rd_kafka_offset_store(rk, rko->rko_offset); /* Destroy the op */ rd_kafka_op_destroy(rk, rko); now = rd_clock(); if (cnt.t_last + dispintvl <= now && cnt.t_start + 1000000 < now) { printf("%% %"PRIu64" messages and %"PRIu64 " bytes: %"PRIu64" msgs/s and " "%.2f Mb/s\n", cnt.msgs, cnt.bytes, (cnt.msgs / ((now - cnt.t_start)/1000)) * 1000, (float)(cnt.bytes / ((now - cnt.t_start) / 1000))); cnt.t_last = now; } } cnt.t_end = rd_clock(); /* Destroy the handle */ rd_kafka_destroy(rk); dirstr = "received"; } if (cnt.t_end_send) cnt.t_total = cnt.t_end_send - cnt.t_start; else cnt.t_total = cnt.t_end - cnt.t_start; printf("%% %"PRIu64" messages and %"PRIu64" bytes " "%s in %"PRIu64"ms: %"PRIu64" msgs/s and %.02f Mb/s\n", cnt.msgs, cnt.bytes, dirstr, cnt.t_total / 1000, (cnt.msgs / (cnt.t_total / 1000)) * 1000, (float)(cnt.bytes / (cnt.t_total / 1000))); if (cnt.t_latency) printf("%% Average application fetch latency: %"PRIu64"us\n", cnt.t_latency / cnt.msgs); return 0; }
int rd_kafka_q_serve_rkmessages (rd_kafka_q_t *rkq, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size) { unsigned int cnt = 0; TAILQ_HEAD(, rd_kafka_op_s) tmpq = TAILQ_HEAD_INITIALIZER(tmpq); rd_kafka_op_t *rko, *next; rd_kafka_t *rk = rkq->rkq_rk; mtx_lock(&rkq->rkq_lock); if (rkq->rkq_fwdq) { rd_kafka_q_t *fwdq = rkq->rkq_fwdq; rd_kafka_q_keep(fwdq); /* Since the q_pop may block we need to release the parent * queue's lock. */ mtx_unlock(&rkq->rkq_lock); cnt = rd_kafka_q_serve_rkmessages(fwdq, timeout_ms, rkmessages, rkmessages_size); rd_kafka_q_destroy(fwdq); return cnt; } mtx_unlock(&rkq->rkq_lock); while (cnt < rkmessages_size) { mtx_lock(&rkq->rkq_lock); while (!(rko = TAILQ_FIRST(&rkq->rkq_q))) { if (cnd_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) == thrd_timedout) break; } if (!rko) { mtx_unlock(&rkq->rkq_lock); break; /* Timed out */ } rd_kafka_q_deq0(rkq, rko); mtx_unlock(&rkq->rkq_lock); if (rd_kafka_op_version_outdated(rko, 0)) { /* Outdated op, put on discard queue */ TAILQ_INSERT_TAIL(&tmpq, rko, rko_link); continue; } /* Serve callbacks */ if (rd_kafka_poll_cb(rk, rko, _Q_CB_CONSUMER, NULL)) { /* Callback served, rko is done, put on discard queue */ TAILQ_INSERT_TAIL(&tmpq, rko, rko_link); continue; } /* Auto-commit offset, if enabled. */ if (!rko->rko_err && rko->rko_type == RD_KAFKA_OP_FETCH) { rd_kafka_toppar_t *rktp; rktp = rd_kafka_toppar_s2i(rko->rko_rktp); rd_kafka_toppar_lock(rktp); rktp->rktp_app_offset = rko->rko_u.fetch.rkm.rkm_offset+1; if (rktp->rktp_cgrp && rk->rk_conf.enable_auto_offset_store) rd_kafka_offset_store0(rktp, rktp->rktp_app_offset, 0/* no lock */); rd_kafka_toppar_unlock(rktp); } /* Get rkmessage from rko and append to array. */ rkmessages[cnt++] = rd_kafka_message_get(rko); } /* Discard non-desired and already handled ops */ next = TAILQ_FIRST(&tmpq); while (next) { rko = next; next = TAILQ_NEXT(next, rko_link); rd_kafka_op_destroy(rko); } return cnt; }
/** * Pop all available ops from a queue and call the provided * callback for each op. * `max_cnt` limits the number of ops served, 0 = no limit. * * Returns the number of ops served. * * Locality: any thread. */ int rd_kafka_q_serve (rd_kafka_q_t *rkq, int timeout_ms, int max_cnt, int cb_type, int (*callback) (rd_kafka_t *rk, rd_kafka_op_t *rko, int cb_type, void *opaque), void *opaque) { rd_kafka_t *rk = rkq->rkq_rk; rd_kafka_op_t *rko; rd_kafka_q_t localq; int cnt = 0; int handled = 0; mtx_lock(&rkq->rkq_lock); rd_dassert(TAILQ_EMPTY(&rkq->rkq_q) || rkq->rkq_qlen > 0); if (rkq->rkq_fwdq) { rd_kafka_q_t *fwdq = rkq->rkq_fwdq; int ret; rd_kafka_q_keep(fwdq); /* Since the q_pop may block we need to release the parent * queue's lock. */ mtx_unlock(&rkq->rkq_lock); ret = rd_kafka_q_serve(fwdq, timeout_ms, max_cnt, cb_type, callback, opaque); rd_kafka_q_destroy(fwdq); return ret; } if (timeout_ms == RD_POLL_INFINITE) timeout_ms = INT_MAX; /* Wait for op */ while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) && timeout_ms != 0) { if (cnd_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) != thrd_success) break; timeout_ms = 0; } if (!rko) { mtx_unlock(&rkq->rkq_lock); return 0; } /* Move the first `max_cnt` ops. */ rd_kafka_q_init(&localq, rkq->rkq_rk); rd_kafka_q_move_cnt(&localq, rkq, max_cnt == 0 ? -1/*all*/ : max_cnt, 0/*no-locks*/); mtx_unlock(&rkq->rkq_lock); rd_kafka_yield_thread = 0; /* Call callback for each op */ while ((rko = TAILQ_FIRST(&localq.rkq_q))) { handled += callback(rk, rko, cb_type, opaque); rd_kafka_q_deq0(&localq, rko); rd_kafka_op_destroy(rko); cnt++; if (unlikely(rd_kafka_yield_thread)) { /* Callback called rd_kafka_yield(), we must * stop our callback dispatching and put the * ops in localq back on the original queue head. */ if (!TAILQ_EMPTY(&localq.rkq_q)) rd_kafka_q_prepend(rkq, &localq); break; } } /* Make sure no op was left unhandled. i.e., * a consumer op ended up on the global queue. */ rd_kafka_assert(NULL, handled == cnt); rd_kafka_q_destroy(&localq); return cnt; }
/** * Serve q like rd_kafka_q_serve() until an op is found that can be returned * as an event to the application. * * @returns the first event:able op, or NULL on timeout. * * Locality: any thread */ rd_kafka_op_t *rd_kafka_q_pop_serve (rd_kafka_q_t *rkq, int timeout_ms, int32_t version, int cb_type, int (*callback) (rd_kafka_t *rk, rd_kafka_op_t *rko, int cb_type, void *opaque), void *opaque) { rd_kafka_op_t *rko; if (timeout_ms == RD_POLL_INFINITE) timeout_ms = INT_MAX; mtx_lock(&rkq->rkq_lock); if (!rkq->rkq_fwdq) { do { /* Filter out outdated ops */ while ((rko = TAILQ_FIRST(&rkq->rkq_q)) && !(rko = rd_kafka_op_filter(rkq, rko, version))) ; if (rko) { int handled; /* Proper versioned op */ rd_kafka_q_deq0(rkq, rko); /* Ops with callbacks are considered handled * and we move on to the next op, if any. * Ops w/o callbacks are returned immediately */ if (callback) { handled = callback(rkq->rkq_rk, rko, cb_type, opaque); if (handled) { rd_kafka_op_destroy(rko); rko = NULL; } } else handled = 0; if (!handled) break; } /* No op, wait for one */ rd_ts_t pre = rd_clock(); if (cnd_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) == thrd_timedout) { mtx_unlock(&rkq->rkq_lock); return NULL; } /* Remove spent time */ timeout_ms -= (int) (rd_clock()-pre) / 1000; if (timeout_ms < 0) timeout_ms = RD_POLL_NOWAIT; } while (timeout_ms != RD_POLL_NOWAIT); mtx_unlock(&rkq->rkq_lock); } else { rd_kafka_q_t *fwdq = rkq->rkq_fwdq; rd_kafka_q_keep(fwdq); /* Since the q_pop may block we need to release the parent * queue's lock. */ mtx_unlock(&rkq->rkq_lock); rko = rd_kafka_q_pop_serve(fwdq, timeout_ms, version, cb_type, callback, opaque); rd_kafka_q_destroy(fwdq); } return rko; }
int main (int argc, char **argv) { rd_kafka_t *rk; char *broker = NULL; char mode = 'C'; char *topic = NULL; int partition = 0; int opt; while ((opt = getopt(argc, argv, "PCt:p:b:")) != -1) { switch (opt) { case 'P': case 'C': mode = opt; break; case 't': topic = optarg; break; case 'p': partition = atoi(optarg); break; case 'b': broker = optarg; break; default: goto usage; } } if (!topic || optind != argc) { usage: fprintf(stderr, "Usage: %s [-C|-P] -t <topic> " "[-p <partition>] [-b <broker>]\n" "\n" " Options:\n" " -C | -P Consumer or Producer mode\n" " -t <topic> Topic to fetch / produce\n" " -p <num> Partition (defaults to 0)\n" " -b <broker> Broker address (localhost:9092)\n" "\n" " In Consumer mode:\n" " writes fetched messages to stdout\n" " In Producer mode:\n" " reads messages from stdin and sends to broker\n" "\n", argv[0]); exit(1); } signal(SIGINT, stop); if (mode == 'P') { /* * Producer */ char buf[2048]; int sendcnt = 0; /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_PRODUCER, broker, NULL))) { perror("kafka_new producer"); exit(1); } fprintf(stderr, "%% Type stuff and hit enter to send\n"); while (run && (fgets(buf, sizeof(buf), stdin))) { int len = strlen(buf); char *opbuf = malloc(len + 1); strncpy(opbuf, buf, len + 1); /* Send/Produce message. */ rd_kafka_produce(rk, topic, partition, RD_KAFKA_OP_F_FREE, opbuf, len); fprintf(stderr, "%% Sent %i bytes to topic " "%s partition %i\n", len, topic, partition); sendcnt++; } /* Wait for messaging to finish. */ while (rd_kafka_outq_len(rk) > 0) usleep(50000); /* Since there is no ack for produce messages in 0.7 * we wait some more for any packets to be sent. * This is fixed in protocol version 0.8 */ if (sendcnt > 0) usleep(500000); /* Destroy the handle */ rd_kafka_destroy(rk); } else if (mode == 'C') { /* * Consumer */ rd_kafka_op_t *rko; /* Base our configuration on the default config. */ rd_kafka_conf_t conf = rd_kafka_defaultconf; /* The offset storage file is optional but its presence * avoids starting all over from offset 0 again when * the program restarts. * ZooKeeper functionality will be implemented in future * versions and then the offset will be stored there instead. */ conf.consumer.offset_file = "."; /* current directory */ /* Indicate to rdkafka that the application is responsible * for storing the offset. This allows the application to * succesfully handle a message before storing the offset. * If this flag is not set rdkafka will store the offset * just prior to returning the message from rd_kafka_consume(). */ conf.flags |= RD_KAFKA_CONF_F_APP_OFFSET_STORE; /* Use the consumer convenience function * to create a Kafka handle. */ if (!(rk = rd_kafka_new_consumer(broker, topic, (uint32_t)partition, 0, &conf))) { perror("kafka_new_consumer"); exit(1); } while (run) { /* Fetch an "op" which is one of: * - a kafka message (if rko_len>0 && rko_err==0) * - an error (if rko_err) */ if (!(rko = rd_kafka_consume(rk, 1000/*timeout ms*/))) continue; if (rko->rko_err) fprintf(stderr, "%% Error: %.*s\n", rko->rko_len, rko->rko_payload); else if (rko->rko_len) { fprintf(stderr, "%% Message with " "next-offset %"PRIu64" is %i bytes\n", rko->rko_offset, rko->rko_len); hexdump(stdout, "Message", rko->rko_payload, rko->rko_len); } /* rko_offset contains the offset of the _next_ * message. We store it when we're done processing * the current message. */ if (rko->rko_offset) rd_kafka_offset_store(rk, rko->rko_offset); /* Destroy the op */ rd_kafka_op_destroy(rk, rko); } /* Destroy the handle */ rd_kafka_destroy(rk); } return 0; }
int main (int argc, char **argv) { rd_kafka_t *rk; char *broker = NULL; int mode_p = 0; int mode_c = 0; char *topic; int partition; /* Command line argument option definition. */ rd_opt_t opts[] = { { RD_OPT_BOOL|RD_OPT_MUT1|RD_OPT_REQ, 'P', "produce", 0, &mode_p, "Run as producer" }, { RD_OPT_BOOL|RD_OPT_MUT1|RD_OPT_REQ, 'C', "consume", 0, &mode_c, "Run as consumer" }, { RD_OPT_STR|RD_OPT_REQ, 't', "topic", 1, &topic, "Topic" }, { RD_OPT_INT|RD_OPT_REQ, 'p', "partition", 1, &partition, "Partition" }, { RD_OPT_STR, 'b', "broker", 1, &broker, "Broker host:port" }, { RD_OPT_END }, }; /* Initialize librd */ rd_init(); /* Parse command line arguments. */ if (!rd_opt_get(opts, argc, argv, NULL, NULL)) exit(1); signal(SIGINT, stop); if (mode_p) { /* * Producer */ char buf[1024]; /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_PRODUCER, broker, NULL))) { perror("kafka_new producer"); exit(1); } fprintf(stderr, "%% Type stuff and hit enter to send\n"); while (run && (fgets(buf, sizeof(buf), stdin))) { int len = strlen(buf); /* Send/Produce message. */ rd_kafka_produce(rk, topic, partition, 0, buf, len); fprintf(stderr, "%% Sent %i bytes to topic " "%s partition %i\n", len, topic, partition); } /* Destroy the handle */ rd_kafka_destroy(rk); } else { /* * Consumer */ rd_kafka_op_t *rko; /* Base our configuration on the default config. */ rd_kafka_conf_t conf = rd_kafka_defaultconf; /* The offset storage file is optional but its presence * avoids starting all over from offset 0 again when * the program restarts. * ZooKeeper functionality will be implemented in future * versions and then the offset will be stored there instead. */ conf.consumer.offset_file = "."; /* current directory */ /* Indicate to rdkafka that the application is responsible * for storing the offset. This allows the application to * succesfully handle a message before storing the offset. * If this flag is not set rdkafka will store the offset * just prior to returning the message from rd_kafka_consume(). */ conf.flags |= RD_KAFKA_CONF_F_APP_OFFSET_STORE; /* Use the consumer convenience function * to create a Kafka handle. */ if (!(rk = rd_kafka_new_consumer(broker, topic, (uint32_t)partition, 0, &conf))) { perror("kafka_new_consumer"); exit(1); } while (run) { /* Fetch an "op" which is one of: * - a kafka message (if rko_len>0 && rko_err==0) * - an error (if rko_err) */ if (!(rko = rd_kafka_consume(rk, 1000/*timeout ms*/))) continue; if (rko->rko_err) fprintf(stderr, "%% Error: %.*s\n", rko->rko_len, rko->rko_payload); else if (rko->rko_len) { fprintf(stderr, "%% Message with " "next-offset %"PRIu64" is %i bytes\n", rko->rko_offset, rko->rko_len); rd_hexdump(stdout, "Message", rko->rko_payload, rko->rko_len); } /* rko_offset contains the offset of the _next_ * message. We store it when we're done processing * the current message. */ if (rko->rko_offset) rd_kafka_offset_store(rk, rko->rko_offset); /* Destroy the op */ rd_kafka_op_destroy(rk, rko); } /* Destroy the handle */ rd_kafka_destroy(rk); } return 0; }