void rd_kafka_op_destroy (rd_kafka_op_t *rko) { /* Decrease refcount on rkbuf to eventually rd_free the shared buffer*/ if (rko->rko_rkbuf) rd_kafka_buf_handle_op(rko, RD_KAFKA_RESP_ERR__DESTROY); else if (rko->rko_payload && rko->rko_flags & RD_KAFKA_OP_F_FREE) { if (rko->rko_free_cb) rko->rko_free_cb(rko->rko_payload); else rd_free(rko->rko_payload); } if (rko->rko_rkt) rd_kafka_topic_destroy0(rd_kafka_topic_a2s(rko->rko_rkt)); if (rko->rko_rktp) rd_kafka_toppar_destroy(rko->rko_rktp); if (rko->rko_metadata) rd_kafka_metadata_destroy(rko->rko_metadata); if (rko->rko_replyq) rd_kafka_q_destroy(rko->rko_replyq); if (rd_atomic32_sub(&rd_kafka_op_cnt, 1) < 0) rd_kafka_assert(NULL, !*"rd_kafka_op_cnt < 0"); rd_free(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 *repq = NULL; rd_kafka_replyq_t rq = RD_KAFKA_NO_REPLYQ; if (!(rkcg = rd_kafka_cgrp_get(rk))) return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP; if (!async) repq = rd_kafka_q_new(rk); if (!async) rq = RD_KAFKA_REPLYQ(repq, 0); err = rd_kafka_commit0(rk, offsets, NULL, rq, NULL, NULL); if (!err && !async) { err = rd_kafka_q_wait_result(repq, RD_POLL_INFINITE); rd_kafka_q_destroy(repq); } return err; }
rd_kafka_queue_t *rd_kafka_queue_new (rd_kafka_t *rk) { rd_kafka_q_t *rkq; rd_kafka_queue_t *rkqu; rkq = rd_kafka_q_new(rk); rkqu = rd_kafka_queue_new0(rk, rkq); rd_kafka_q_destroy(rkq); /* Loose refcount from q_new, one is held * by queue_new0 */ return rkqu; }
void rd_kafka_queue_destroy (rd_kafka_queue_t *rkqu) { int do_destroy; mtx_lock(&rkqu->rkqu_q.rkq_lock); do_destroy = rkqu->rkqu_q.rkq_refcnt == 1; mtx_unlock(&rkqu->rkqu_q.rkq_lock); rd_kafka_q_destroy(&rkqu->rkqu_q); if (!do_destroy) return; /* Still references */ rd_free(rkqu); }
/** * Pop an op from a queue. * * Locality: any thread. */ rd_kafka_op_t *rd_kafka_q_pop (rd_kafka_q_t *rkq, int timeout_ms, int32_t version) { 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))) ; if (rko) { /* Proper versioned op */ rd_kafka_q_deq0(rkq, rko); 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(fwdq, timeout_ms, version); rd_kafka_q_destroy(fwdq); } return rko; }
/** * Send request to queue, wait for response. * Creates a temporary reply queue. */ rd_kafka_op_t *rd_kafka_op_req (rd_kafka_q_t *destq, rd_kafka_op_t *rko, int timeout_ms) { rd_kafka_q_t *recvq; rd_kafka_op_t *reply; recvq = rd_kafka_q_new(destq->rkq_rk); reply = rd_kafka_op_req0(destq, recvq, rko, timeout_ms); rd_kafka_q_destroy(recvq); return reply; }
/** * Set/clear forward queue. * Queue forwarding enables message routing inside rdkafka. * Typical use is to re-route all fetched messages for all partitions * to one single queue. * * All access to rkq_fwdq are protected by rkq_lock. */ void rd_kafka_q_fwd_set0 (rd_kafka_q_t *srcq, rd_kafka_q_t *destq, int do_lock) { if (do_lock) mtx_lock(&srcq->rkq_lock); if (srcq->rkq_fwdq) { rd_kafka_q_destroy(srcq->rkq_fwdq); srcq->rkq_fwdq = NULL; } if (destq) { rd_kafka_q_keep(destq); srcq->rkq_fwdq = destq; /* If rkq has ops in queue, append them to fwdq's queue. * This is an irreversible operation. */ if (srcq->rkq_qlen > 0) rd_kafka_q_concat(destq, srcq); } if (do_lock) mtx_unlock(&srcq->rkq_lock); }
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; }
/** * 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; }
/** * 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 ? NULL : tmpq, NULL); if (!async) { err = rd_kafka_q_wait_result(tmpq, RD_POLL_INFINITE); rd_kafka_q_destroy(tmpq); } else { err = RD_KAFKA_RESP_ERR_NO_ERROR; } return err; }
void rd_kafka_queue_destroy (rd_kafka_queue_t *rkqu) { rd_kafka_q_disable(rkqu->rkqu_q); rd_kafka_q_destroy(rkqu->rkqu_q); rd_free(rkqu); }
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; }