/** * Store offset. * Typically called from application code. * * NOTE: No lucks must be held. */ rd_kafka_resp_err_t rd_kafka_offset_store (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset) { rd_kafka_toppar_t *rktp; /* Find toppar */ rd_kafka_topic_rdlock(rkt); if (!(rktp = rd_kafka_toppar_get(rkt, partition, 0/*!ua_on_miss*/))) { rd_kafka_topic_unlock(rkt); return RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; } rd_kafka_topic_unlock(rkt); rd_kafka_offset_store0(rktp, offset, 1/*lock*/); rd_kafka_toppar_destroy(rktp); return RD_KAFKA_RESP_ERR_NO_ERROR; }
/** * @brief Store offset for fetched message. */ void rd_kafka_op_offset_store (rd_kafka_t *rk, rd_kafka_op_t *rko, const rd_kafka_message_t *rkmessage) { rd_kafka_toppar_t *rktp; if (unlikely(rko->rko_type != RD_KAFKA_OP_FETCH || rko->rko_err)) return; rktp = rd_kafka_toppar_s2i(rko->rko_rktp); if (unlikely(!rk)) rk = rktp->rktp_rkt->rkt_rk; rd_kafka_toppar_lock(rktp); rktp->rktp_app_offset = rkmessage->offset+1; if (rk->rk_conf.enable_auto_offset_store) rd_kafka_offset_store0(rktp, rkmessage->offset+1, 0/*no lock*/); rd_kafka_toppar_unlock(rktp); }
/** * Terminates toppar's offset store. * NOTE: toppar_lock(rktp) must be held. */ void rd_kafka_offset_store_term (rd_kafka_toppar_t *rktp) { if (!(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE)) return; if (rktp->rktp_rkt->rkt_conf.auto_commit && rktp->rktp_stored_offset < rktp->rktp_eof_offset) rd_kafka_offset_store0(rktp, rktp->rktp_eof_offset, 0/*no lock*/); switch (rktp->rktp_rkt->rkt_conf.offset_store_method) { case RD_KAFKA_OFFSET_METHOD_FILE: rd_kafka_offset_file_term(rktp); break; case RD_KAFKA_OFFSET_METHOD_BROKER: rd_kafka_offset_broker_term(rktp); break; } }
/** * Stop toppar's offset store, committing the final offsets, etc. * * Returns RD_KAFKA_RESP_ERR_NO_ERROR on success, * RD_KAFKA_RESP_ERR__IN_PROGRESS if the term triggered an * async operation (e.g., broker offset commit), or * any other error in case of immediate failure. * * The offset layer will call rd_kafka_offset_store_term() when * the offset management has been fully stopped for this partition. * * Locks: rd_kafka_toppar_lock() MUST be held. */ rd_kafka_resp_err_t rd_kafka_offset_store_stop (rd_kafka_toppar_t *rktp) { rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; if (!(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE)) goto done; rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_OFFSET_STORE_STOPPING; rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "%s [%"PRId32"]: stopping offset store " "(stored offset %"PRId64 ", committed offset %"PRId64", EOF offset %"PRId64")", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_stored_offset, rktp->rktp_committed_offset, rktp->rktp_offsets_fin.eof_offset); /* Store end offset for empty partitions */ if (((!rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk) && rktp->rktp_rkt->rkt_rk->rk_conf.enable_auto_commit) || rktp->rktp_rkt->rkt_conf.auto_commit) && rktp->rktp_stored_offset == -1 && rktp->rktp_offsets_fin.eof_offset > 0) rd_kafka_offset_store0(rktp, rktp->rktp_offsets_fin.eof_offset, 0/*no lock*/); /* Commit offset to backing store. * This might be an async operation. */ if (rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk) && rktp->rktp_stored_offset > rktp->rktp_committed_offset) err = rd_kafka_offset_commit(rktp); /* If stop is in progress (async commit), return now. */ if (err == RD_KAFKA_RESP_ERR__IN_PROGRESS) return err; done: /* Stop is done */ rd_kafka_offset_store_term(rktp, err); return RD_KAFKA_RESP_ERR_NO_ERROR; }
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; }