diff --git a/CONFIGURATION.md b/CONFIGURATION.md index 40b7412efd..ddf795a9e2 100644 --- a/CONFIGURATION.md +++ b/CONFIGURATION.md @@ -154,6 +154,7 @@ offset_commit_cb | C | | enable.partition.eof | C | true, false | false | low | Emit RD_KAFKA_RESP_ERR__PARTITION_EOF event whenever the consumer reaches the end of a partition.
*Type: boolean* check.crcs | C | true, false | false | medium | Verify CRC32 of consumed messages, ensuring no on-the-wire or on-disk corruption to the messages occurred. This check comes at slightly increased CPU usage.
*Type: boolean* client.rack | * | | | low | A rack identifier for this client. This can be any string value which indicates where this client is physically located. It corresponds with the broker config `broker.rack`.
*Type: string* +max.poll.records | C | 1 .. 2147483647 | 500 | low | tba description,
*Type: integer* transactional.id | P | | | high | Enables the transactional producer. The transactional.id is used to identify the same transactional producer instance across process restarts. It allows the producer to guarantee that transactions corresponding to earlier instances of the same producer have been finalized prior to starting any new transactions, and that any zombie instances are fenced off. If no transactional.id is provided, then the producer is limited to idempotent delivery (if enable.idempotence is set). Requires broker version >= 0.11.0.
*Type: string* transaction.timeout.ms | P | 1000 .. 2147483647 | 60000 | medium | The maximum amount of time in milliseconds that the transaction coordinator will wait for a transaction status update from the producer before proactively aborting the ongoing transaction. If this value is larger than the `transaction.max.timeout.ms` setting in the broker, the init_transactions() call will fail with ERR_INVALID_TRANSACTION_TIMEOUT. The transaction timeout automatically adjusts `message.timeout.ms` and `socket.timeout.ms`, unless explicitly configured in which case they must not exceed the transaction timeout (`socket.timeout.ms` must be at least 100ms lower than `transaction.timeout.ms`). This is also the default timeout value if no timeout (-1) is supplied to the transactional API methods.
*Type: integer* enable.idempotence | P | true, false | false | high | When set to `true`, the producer will ensure that messages are successfully produced exactly once and in the original produce order. The following configuration properties are adjusted automatically (if not modified by the user) when idempotence is enabled: `max.in.flight.requests.per.connection=5` (must be less than or equal to 5), `retries=INT32_MAX` (must be greater than 0), `acks=all`, `queuing.strategy=fifo`. Producer instantation will fail if user-supplied configuration is incompatible.
*Type: boolean* diff --git a/examples/consumer.c b/examples/consumer.c index f621a9db35..993a01ca5a 100644 --- a/examples/consumer.c +++ b/examples/consumer.c @@ -33,10 +33,15 @@ * (https://github.com/confluentinc/librdkafka) */ +#ifndef _POSIX_C_SOURCE +#define _POSIX_C_SOURCE 199309L +#endif + #include #include #include #include +#include /* Typical include path would be , but this program @@ -149,6 +154,29 @@ int main(int argc, char **argv) { return 1; } + if (rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)) != RD_KAFKA_CONF_OK) { + fprintf(stderr, "%s\n", errstr); + rd_kafka_conf_destroy(conf); + return 1; + } + + + if (rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)) != RD_KAFKA_CONF_OK) { + fprintf(stderr, "%s\n", errstr); + rd_kafka_conf_destroy(conf); + return 1; + } + + + // if (rd_kafka_conf_set(conf, "debug", "all", errstr, sizeof(errstr)) != + // RD_KAFKA_CONF_OK) { + // fprintf(stderr, "%s\n", errstr); + // rd_kafka_conf_destroy(conf); + // return 1; + // } + /* * Create consumer instance. * @@ -213,49 +241,68 @@ int main(int argc, char **argv) { * since a rebalance may happen at any time. * Start polling for messages. */ + rd_kafka_message_t *rkmessages[500]; while (run) { - rd_kafka_message_t *rkm; - - rkm = rd_kafka_consumer_poll(rk, 100); - if (!rkm) - continue; /* Timeout: no message within 100ms, - * try again. This short timeout allows - * checking for `run` at frequent intervals. - */ - - /* consumer_poll() will return either a proper message - * or a consumer error (rkm->err is set). */ - if (rkm->err) { - /* Consumer errors are generally to be considered - * informational as the consumer will automatically - * try to recover from all types of errors. */ - fprintf(stderr, "%% Consumer error: %s\n", - rd_kafka_message_errstr(rkm)); - rd_kafka_message_destroy(rkm); + rd_kafka_message_t *rkm = NULL; + size_t rcvd_msgs = 0; + int i; + + // rkm = rd_kafka_consumer_poll(rk, 100); + rd_kafka_error_t *error; + + // fprintf(stderr, "Calling consume_batch\n"); + struct timespec __t0, __t1; + if (clock_gettime(CLOCK_MONOTONIC, &__t0) != 0) + perror("clock_gettime"); + error = rd_kafka_share_consume_batch(rk, 500, rkmessages, &rcvd_msgs); + if (clock_gettime(CLOCK_MONOTONIC, &__t1) != 0) + perror("clock_gettime"); + double __elapsed_ms = + (__t1.tv_sec - __t0.tv_sec) * 1000.0 + (__t1.tv_nsec - __t0.tv_nsec) / 1e6; + // fprintf(stdout, "%% rd_kafka_share_consume_batch() took %.3f ms\n", __elapsed_ms); + + if (error) { + fprintf(stderr, "%% Consume error: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); continue; } - /* Proper message. */ - printf("Message on %s [%" PRId32 "] at offset %" PRId64 - " (leader epoch %" PRId32 "):\n", - rd_kafka_topic_name(rkm->rkt), rkm->partition, - rkm->offset, rd_kafka_message_leader_epoch(rkm)); - - /* Print the message key. */ - if (rkm->key && is_printable(rkm->key, rkm->key_len)) - printf(" Key: %.*s\n", (int)rkm->key_len, - (const char *)rkm->key); - else if (rkm->key) - printf(" Key: (%d bytes)\n", (int)rkm->key_len); - - /* Print the message value/payload. */ - if (rkm->payload && is_printable(rkm->payload, rkm->len)) - printf(" Value: %.*s\n", (int)rkm->len, - (const char *)rkm->payload); - else if (rkm->payload) - printf(" Value: (%d bytes)\n", (int)rkm->len); - - rd_kafka_message_destroy(rkm); + // fprintf(stderr, "%% Received %zu messages\n", rcvd_msgs); + for (i = 0; i < (int)rcvd_msgs; i++) { + rkm = rkmessages[i]; + + if (rkm->err) { + fprintf(stderr, "%% Consumer error: %d: %s\n", + rkm->err, rd_kafka_message_errstr(rkm)); + rd_kafka_message_destroy(rkm); + continue; + } + + // if((int)rcvd_msgs < -1) { + /* Proper message. */ + printf("Message received on %s [%" PRId32 "] at offset %" PRId64, + rd_kafka_topic_name(rkm->rkt), rkm->partition, + rkm->offset); + + /* Print the message key. */ + if (rkm->key && is_printable(rkm->key, rkm->key_len)) + printf(" Key: %.*s\n", (int)rkm->key_len, + (const char *)rkm->key); + else if (rkm->key) + printf(" Key: (%d bytes)\n", (int)rkm->key_len); + + /* Print the message value/payload. */ + if (rkm->payload && + is_printable(rkm->payload, rkm->len)) + printf(" - Value: %.*s\n", (int)rkm->len, + (const char *)rkm->payload); + else if (rkm->payload) + printf(" - Value: (%d bytes)\n", (int)rkm->len); + // } + + rd_kafka_message_destroy(rkm); + } } diff --git a/src/rd.h b/src/rd.h index 300a7b030c..aa2f4f483b 100644 --- a/src/rd.h +++ b/src/rd.h @@ -406,14 +406,17 @@ static RD_INLINE RD_UNUSED int rd_refcnt_get(rd_refcnt_t *R) { rd_refcnt_get(R), (R), WHAT, __FUNCTION__, __LINE__), \ rd_refcnt_sub0(R)) -#define rd_refcnt_sub(R) \ +#define rd_refcnt_sub_fl(FUNC, LINE, R) \ (fprintf(stderr, "REFCNT DEBUG: %-35s %d -1: %16p: %s:%d\n", #R, \ - rd_refcnt_get(R), (R), __FUNCTION__, __LINE__), \ + rd_refcnt_get(R), (R), (FUNC), (LINE)), \ rd_refcnt_sub0(R)) +#define rd_refcnt_sub(R) rd_refcnt_sub_fl(__FUNCTION__, __LINE__, R) + #else #define rd_refcnt_add_fl(FUNC, LINE, R) rd_refcnt_add0(R) #define rd_refcnt_add(R) rd_refcnt_add0(R) +#define rd_refcnt_sub_fl(FUNC, LINE, R) rd_refcnt_sub0(R) #define rd_refcnt_sub(R) rd_refcnt_sub0(R) #endif diff --git a/src/rdkafka.c b/src/rdkafka.c index c6f89ad469..d69b749541 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -2839,6 +2839,410 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type, return NULL; } +rd_kafka_t *rd_kafka_share_consumer_new(rd_kafka_conf_t *conf, + char *errstr, + size_t errstr_size) { + rd_kafka_t *rk; + char errstr_internal[512]; + rd_kafka_conf_res_t res; + + if (conf == NULL) { + rd_snprintf(errstr, errstr_size, + "rd_kafka_share_consumer_new(): " + "conf argument must not be NULL"); + return NULL; + } + + res = rd_kafka_conf_set(conf, "share.consumer", "true", errstr_internal, + sizeof(errstr_internal)); + if (res != RD_KAFKA_CONF_OK) { + rd_snprintf(errstr, errstr_size, + "rd_kafka_share_consumer_new(): " + "Failed to set share.consumer=true: %s", + errstr_internal); + return NULL; + } + + + res = rd_kafka_conf_set(conf, "group.protocol", "consumer", + errstr_internal, sizeof(errstr_internal)); + if (res != RD_KAFKA_CONF_OK) { + rd_snprintf(errstr, errstr_size, + "rd_kafka_share_consumer_new(): " + "Failed to set group.protocol=consumer: %s", + errstr_internal); + return NULL; + } + + rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, errstr_size); + if (!rk) { + /* If rd_kafka_new() failed it will have set the last error + * and filled out errstr, so we don't need to do that here. */ + return NULL; + } + return rk; +} + + +/** + * @locality main thread + * @locks none + */ +static rd_kafka_broker_t *rd_kafka_share_select_broker(rd_kafka_t *rk, + rd_kafka_cgrp_t *rkcg) { + rd_kafka_broker_t *selected_rkb = NULL; + rd_kafka_topic_partition_list_t *partitions = + rkcg->rkcg_current_assignment; + // rkcg->rkcg_toppars; /* TODO: use rkcg->rkcg_toppars instead. */ + size_t i; + rd_kafka_topic_partition_t *partition; + + if (!partitions || partitions->cnt == 0) { + rd_kafka_dbg(rk, CGRP, "SHARE", + "No partitions assigned to consumer, " + "cannot select broker for share fetch"); + return NULL; + } + + /* Look through all partitions in order, find the first one which + * has a leader. */ + rd_kafka_dbg(rk, CGRP, "SHARE", + "Selecting broker for share fetch from %d assigned " + "partitions, last picked index = %" PRIusz, + partitions->cnt, rkcg->rkcg_share.last_partition_picked); + + for (i = 0; i < (size_t)partitions->cnt; i++) { + rd_kafka_toppar_t *rktp; + rkcg->rkcg_share.last_partition_picked += 1; + if (rkcg->rkcg_share.last_partition_picked >= + (size_t)partitions->cnt) + rkcg->rkcg_share.last_partition_picked = 0; + partition = + &partitions->elems[rkcg->rkcg_share.last_partition_picked]; + + rktp = rd_kafka_toppar_get2(rk, partition->topic, + partition->partition, 0, 1); + + /* Criteria to choose a broker: + * 1. It should be the leader of a partition. + * 2. A share-fetch op must not already be enqueued on it. */ + if (rktp->rktp_leader) { + /* TODO: We're only going to access + * rkb_share_fetch_enqueued from the main thread, except + * when it's being calloc'd and destroyed. Is it safe to + * access it without a lock? */ + rd_kafka_broker_lock(rktp->rktp_leader); + if (!rktp->rktp_leader->rkb_share_fetch_enqueued) { + rd_kafka_broker_keep(rktp->rktp_leader); + selected_rkb = rktp->rktp_leader; + } + rd_kafka_broker_unlock(rktp->rktp_leader); + } + + rd_kafka_toppar_destroy(rktp); + + if (selected_rkb) + break; + } + return selected_rkb; +} + +/** + * @brief Timer callback for reenequeing SHARE_FETCH_FANOUT after a backoff. + * @locality main thread + * @locks none + */ +static void rd_kafka_share_fetch_fanout_renqueue(rd_kafka_timers_t *rkts, + void *arg) { + rd_kafka_op_t *rko = arg; + rd_kafka_t *rk = rkts->rkts_rk; + + rd_kafka_dbg(rk, CGRP, "SHARE", "Re-enqueing SHARE_FETCH_FANOUT"); + rd_kafka_q_enq(rk->rk_ops, rko); +} + +/** + * @brief Enqueue a SHARE_FETCH_FANOUT op on the main queue. + * @param backoff_ms If >0 the op will be enqueued after this many milliseconds. + * Else, it will be immediate. + * @locality any thread + */ +static void rd_kafka_share_fetch_fanout_with_backoff(rd_kafka_t *rk, + rd_ts_t abs_timeout, + int backoff_ms) { + rd_kafka_cgrp_t *rkcg = rd_kafka_cgrp_get(rk); + rd_kafka_op_t *rko = rd_kafka_op_new_cb( + rk, RD_KAFKA_OP_SHARE_FETCH_FANOUT, rd_kafka_share_fetch_fanout_op); + rko->rko_u.share_fetch_fanout.abs_timeout = abs_timeout; + rko->rko_replyq = RD_KAFKA_REPLYQ(rk->rk_ops, 0); + + if (backoff_ms > 0) + rd_kafka_timer_start_oneshot( + &rk->rk_timers, &rkcg->rkcg_share.share_fetch_fanout_tmr, + rd_true, backoff_ms * 1000, + rd_kafka_share_fetch_fanout_renqueue, rko); + else + rd_kafka_q_enq(rk->rk_ops, rko); +} + +/** + * Handles RD_KAFKA_OP_SHARE_FETCH | RD_KAFKA_OP_REPLY. + * @locality main thread + */ +rd_kafka_op_res_t rd_kafka_share_fetch_reply_op(rd_kafka_t *rk, + rd_kafka_op_t *rko_orig) { + rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; + + rd_kafka_assert(rk, thrd_is_current(rk->rk_thread)); + rd_kafka_dbg(rk, CGRP, "SHAREFETCH", + "Fetch share reply: %s, should_fetch=%d, broker=%s", + rd_kafka_err2str(rko_orig->rko_err), + rko_orig->rko_u.share_fetch.should_fetch, + rko_orig->rko_u.share_fetch.target_broker + ? rd_kafka_broker_name( + rko_orig->rko_u.share_fetch.target_broker) + : "none"); + + rko_orig->rko_u.share_fetch.target_broker->rkb_share_fetch_enqueued = + rd_false; + + if (!rko_orig->rko_err && !rko_orig->rko_error) + return RD_KAFKA_OP_RES_HANDLED; + + /* Retry fetching if there is any amount of time left. For other + * errors, let them drop and be retried on the next poll. */ + /* TODO: KIP-932: Are there errors for which we should not + * retry, and rather propagate those errors to user? In that + * case, we must write to the rkcg queue. */ + if (!rko_orig->rko_u.share_fetch.should_fetch || + !rd_timeout_remains(rko_orig->rko_u.share_fetch.abs_timeout)) + return RD_KAFKA_OP_RES_HANDLED; + + if (rko_orig->rko_error) + err = rd_kafka_error_code(rko_orig->rko_error); + + switch (err) { + /* For some errors, don't resend at all. + * This list of errors is incomplete. */ + case RD_KAFKA_RESP_ERR__TIMED_OUT: + case RD_KAFKA_RESP_ERR__DESTROY: + case RD_KAFKA_RESP_ERR__UNKNOWN_GROUP: + case RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS: /* This should not happen. */ + case RD_KAFKA_RESP_ERR__STATE: + case RD_KAFKA_RESP_ERR__AUTHENTICATION: + case RD_KAFKA_RESP_ERR_NO_ERROR: + case RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED: /* Do we need more + handling for fatal + errors? */ + break; + + /* For other cases, resend to separate broker after a backoff. */ + default: + rd_kafka_share_fetch_fanout_with_backoff( + rk, rko_orig->rko_u.share_fetch.abs_timeout, + rk->rk_conf.retry_backoff_max_ms); + break; + } + + return RD_KAFKA_OP_RES_HANDLED; +} + +/** + * Handles RD_KAFKA_OP_SHARE_FETCH_FANOUT | RD_KAFKA_OP_REPLY. + * @locality main thread + */ +rd_kafka_op_res_t +rd_kafka_share_fetch_fanout_reply_op(rd_kafka_t *rk, rd_kafka_op_t *rko_orig) { + rd_kafka_resp_err_t err; + + if (!rko_orig->rko_err && !rko_orig->rko_error) + return RD_KAFKA_OP_RES_HANDLED; + + err = rko_orig->rko_err; + if (rko_orig->rko_error) + err = rd_kafka_error_code(rko_orig->rko_error); + + /* TODO: KIP-932: Add error handling - either retries, or user-level + * propagation, later. */ + rd_kafka_dbg( + rk, CGRP, "SHARE", + "Encountered error in SHARE_FETCH_FANOUT: %s, remains: %d", + rd_kafka_err2name(err), + rd_timeout_remains(rko_orig->rko_u.share_fetch_fanout.abs_timeout)); + + switch (err) { + /* Some errors need not be retried. */ + case RD_KAFKA_RESP_ERR__DESTROY: + case RD_KAFKA_RESP_ERR__TIMED_OUT: + break; + + /* Some errors may be retried - with a constant backoff. */ + default: + rd_kafka_share_fetch_fanout_with_backoff( + rk, rko_orig->rko_u.share_fetch_fanout.abs_timeout, + /* TODO: KIP-932: Consider setting this to retry_backoff_ms + or to a constant.*/ + rk->rk_conf.retry_backoff_max_ms); + break; + } + return RD_KAFKA_OP_RES_HANDLED; +} + +/** + * Op callback for RD_KAFKA_OP_SHARE_FETCH_FANOUT. + * @locality main thread + */ +rd_kafka_op_res_t rd_kafka_share_fetch_fanout_op(rd_kafka_t *rk, + rd_kafka_q_t *rkq, + rd_kafka_op_t *rko) { + rd_kafka_broker_t *rkb, *selected_rkb = NULL; + rd_kafka_error_t *error; + rd_kafka_cgrp_t *rkcg = rd_kafka_cgrp_get(rk); + rd_kafka_op_t *reply_rko; + rd_ts_t abs_timeout = rko->rko_u.share_fetch_fanout.abs_timeout; + + /* We should never have enqueued this op if we didn't have the rkcg */ + rd_assert(rkcg); + + if (rd_timeout_remains(abs_timeout) <= 0) { + rd_kafka_dbg(rk, CGRP, "SHARE", + "Not issuing SHARE_FETCH_FANOUT: timeout expired"); + + error = rd_kafka_error_new(RD_KAFKA_RESP_ERR__TIMED_OUT, + "rd_kafka_share_fetch_fanout_op(): " + "Share fetch fanout op timed out"); + + reply_rko = rd_kafka_op_new(RD_KAFKA_OP_SHARE_FETCH_FANOUT | + RD_KAFKA_OP_REPLY); + reply_rko->rko_error = error; + reply_rko->rko_u.share_fetch_fanout.abs_timeout = abs_timeout; + rd_kafka_replyq_enq(&rko->rko_replyq, reply_rko, 0); + return RD_KAFKA_OP_RES_HANDLED; + } + + if (!(selected_rkb = rd_kafka_share_select_broker(rk, rkcg))) { + error = + rd_kafka_error_new(RD_KAFKA_RESP_ERR__STATE, + "rd_kafka_share_consume_batch(): " + "No broker available for share fetch"); + + reply_rko = rd_kafka_op_new(RD_KAFKA_OP_SHARE_FETCH_FANOUT | + RD_KAFKA_OP_REPLY); + reply_rko->rko_error = error; + reply_rko->rko_u.share_fetch_fanout.abs_timeout = abs_timeout; + rd_kafka_replyq_enq(&rko->rko_replyq, reply_rko, 0); + return RD_KAFKA_OP_RES_HANDLED; + } + + /* Issue fetch requests to all brokers */ + rd_kafka_dbg(rk, CGRP, "SHARE", + "Selected broker %s for fetching messages, issuing fetch " + "requests to necessary brokers", + rd_kafka_broker_name(selected_rkb)); + + rd_kafka_rdlock(rk); + TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) { + rd_kafka_op_t *rko_sf; + + if (rd_kafka_broker_or_instance_terminating(rkb) || + RD_KAFKA_BROKER_IS_LOGICAL(rkb)) { + rd_kafka_dbg(rk, CGRP, "SHARE", + "Skipping broker %s for share fetch: " + "terminating or logical", + rd_kafka_broker_name(rkb)); + continue; + } + + if (rkb->rkb_share_fetch_enqueued) { + rd_kafka_broker_unlock(rkb); + rd_kafka_dbg(rk, CGRP, "SHARE", + "Unable to enqueue op on broker %s " + "because another op is already pending.", + rd_kafka_broker_name(rkb)); + continue; + } + rkb->rkb_share_fetch_enqueued = rd_true; + + rko_sf = rd_kafka_op_new(RD_KAFKA_OP_SHARE_FETCH); + rko_sf->rko_u.share_fetch.should_leave = rd_false; + rko_sf->rko_u.share_fetch.abs_timeout = abs_timeout; + rko_sf->rko_u.share_fetch.should_fetch = (rkb == selected_rkb); + rd_kafka_broker_keep(rkb); + rko_sf->rko_u.share_fetch.target_broker = rkb; + rko_sf->rko_replyq = RD_KAFKA_REPLYQ(rk->rk_ops, 0); + + rd_kafka_dbg(rk, CGRP, "SHAREFETCH", + "Enqueuing share fetch op on broker %s " + "(%s leave), (%s fetch)", + rd_kafka_broker_name(rkb), + rko_sf->rko_u.share_fetch.should_leave + ? "should" + : "should not", + rko_sf->rko_u.share_fetch.should_fetch + ? "should" + : "should not"); + rd_kafka_q_enq(rkb->rkb_ops, rko_sf); + } + rd_kafka_rdunlock(rk); + + RD_IF_FREE(selected_rkb, rd_kafka_broker_destroy); + + return RD_KAFKA_OP_RES_HANDLED; +} + +rd_kafka_error_t *rd_kafka_share_consume_batch( + rd_kafka_t *rk, + int timeout_ms, + /* There is some benefit to making this ***rkmessages and allocating it + within this function, but on the flipside this means that it will always + be allocated on the heap. */ + rd_kafka_message_t **rkmessages /* out */, + size_t *rkmessages_size /* out */) { + rd_kafka_cgrp_t *rkcg; + rd_ts_t now = rd_clock(); + rd_ts_t abs_timeout = rd_timeout_init0(now, timeout_ms); + size_t max_poll_records = (size_t)rk->rk_conf.share.max_poll_records; + + if (!RD_KAFKA_IS_SHARE_CONSUMER(rk)) + return rd_kafka_error_new(RD_KAFKA_RESP_ERR__INVALID_ARG, + "rd_kafka_share_consume_batch(): " + "rk is not a shared consumer"); + + if (unlikely(!(rkcg = rd_kafka_cgrp_get(rk)))) + return rd_kafka_error_new(RD_KAFKA_RESP_ERR__STATE, + "rd_kafka_share_consume_batch(): " + "Consumer group not initialized"); + + /* If we have any pending items on the consumer queue, don't issue new + * requests, rather, deal with them first. + * + * TODO KIP-932: + * Above statement might be incorrect as we have to send all the pending + * acknowledgements irrespective of whether there are messages to be + * consumed or not. + */ + if (likely(rd_kafka_q_len(rkcg->rkcg_q) == 0)) { + rd_kafka_dbg(rk, CGRP, "SHARE", + "Issuing share fetch fanout to main thread with " + "abs_timeout = %" PRId64, + abs_timeout); + rd_kafka_share_fetch_fanout_with_backoff(rk, abs_timeout, + 0 /* no backoff */); + } + + /* At this point, there's no reason to deviate from what we already do + * for returning multiple messages to the user, as the orchestration + * is handled by the main thread. Later on, we needed, we might need + * a custom loop if we need any changes. */ + *rkmessages_size = rd_kafka_q_serve_share_rkmessages( + rkcg->rkcg_q, timeout_ms, /* Use this timeout directly as prior + operations aren't blocking, so no need to + re-convert the abs_timeout into a relative one.*/ + rkmessages, max_poll_records); + + return NULL; +} + /** * Schedules a rebootstrap of the cluster immediately. * @@ -4288,6 +4692,15 @@ rd_kafka_op_res_t rd_kafka_poll_cb(rd_kafka_t *rk, res = rd_kafka_metadata_update_op(rk, rko->rko_u.metadata.mdi); break; + case RD_KAFKA_OP_SHARE_FETCH | RD_KAFKA_OP_REPLY: + res = rd_kafka_share_fetch_reply_op(rk, rko); + break; + + case RD_KAFKA_OP_SHARE_FETCH_FANOUT | RD_KAFKA_OP_REPLY: + rd_kafka_assert(rk, thrd_is_current(rk->rk_thread)); + res = rd_kafka_share_fetch_fanout_reply_op(rk, rko); + break; + default: /* If op has a callback set (e.g., OAUTHBEARER_REFRESH), * call it. */ @@ -5459,29 +5872,44 @@ char *rd_kafka_Uuid_str(const rd_kafka_Uuid_t *uuid) { } const char *rd_kafka_Uuid_base64str(const rd_kafka_Uuid_t *uuid) { - if (*uuid->base64str) - return uuid->base64str; - - rd_chariov_t in_base64; - char *out_base64_str; - char *uuid_bytes; - uint64_t input_uuid[2]; - - input_uuid[0] = htobe64(uuid->most_significant_bits); - input_uuid[1] = htobe64(uuid->least_significant_bits); - uuid_bytes = (char *)input_uuid; - in_base64.ptr = uuid_bytes; - in_base64.size = sizeof(uuid->most_significant_bits) + - sizeof(uuid->least_significant_bits); - - out_base64_str = rd_base64_encode_str(&in_base64); - if (!out_base64_str) - return NULL; - - rd_strlcpy((char *)uuid->base64str, out_base64_str, - 23 /* Removing extra ('=') padding */); - rd_free(out_base64_str); + if (*uuid->base64str) return uuid->base64str; + + rd_chariov_t in_base64; + char *out_base64_str; + char *uuid_bytes; + uint64_t input_uuid[2]; + + input_uuid[0] = htobe64(uuid->most_significant_bits); + input_uuid[1] = htobe64(uuid->least_significant_bits); + uuid_bytes = (char *)input_uuid; + in_base64.ptr = uuid_bytes; + in_base64.size = sizeof(uuid->most_significant_bits) + + sizeof(uuid->least_significant_bits); + + // Standard Base64 encode + out_base64_str = rd_base64_encode_str(&in_base64); + if (!out_base64_str) + return NULL; + + // Convert to URL-safe Base64 + for (char *p = out_base64_str; *p; p++) { + if (*p == '+') + *p = '-'; + else if (*p == '/') + *p = '_'; + } + + // Strip '=' padding (Kafka’s Base64 UUIDs are 22 chars) + size_t len = strlen(out_base64_str); + while (len > 0 && out_base64_str[len - 1] == '=') { + out_base64_str[--len] = '\0'; + } + + rd_strlcpy((char *)uuid->base64str, out_base64_str, sizeof(uuid->base64str)); + rd_free(out_base64_str); + + return uuid->base64str; } unsigned int rd_kafka_Uuid_hash(const rd_kafka_Uuid_t *uuid) { diff --git a/src/rdkafka.h b/src/rdkafka.h index 57cf08bad1..42f8bfdb9c 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -3057,6 +3057,27 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type, size_t errstr_size); +RD_EXPORT +rd_kafka_t *rd_kafka_share_consumer_new(rd_kafka_conf_t *conf, + char *errstr, + size_t errstr_size); + +/** + * @brief Consume a batch of messages from the share consumer instance. + * + * @param rk Share consumer instance. + * @param timeout_ms Maximum time to block waiting for messages. + * @param rkmessages Output array of messages - this must be preallocated with + * at least enough capacity for size max.poll.records. + * @param rkmessages_size Output number of messages returned in rkmessages. + */ +RD_EXPORT +rd_kafka_error_t * +rd_kafka_share_consume_batch(rd_kafka_t *rk, + int timeout_ms, + rd_kafka_message_t **rkmessages /* out */, + size_t *rkmessages_size /* out */); + /** * @brief Destroy Kafka handle. * diff --git a/src/rdkafka_broker.c b/src/rdkafka_broker.c index eb8e849240..c8b4522631 100644 --- a/src/rdkafka_broker.c +++ b/src/rdkafka_broker.c @@ -3178,6 +3178,86 @@ static rd_kafka_resp_err_t rd_kafka_broker_destroy_error(rd_kafka_t *rk) { : RD_KAFKA_RESP_ERR__DESTROY_BROKER; } + +/** + * @brief Add description. + * + * @locality broker thread + * @locks toppar lock + * @locks broker lock + */ +static void rd_kafka_broker_share_session_add_remove_toppar(rd_list_t **toppars_add_list, + rd_list_t **toppars_remove_list, + rd_kafka_toppar_t *rktp) { + if (!*toppars_add_list) { + *toppars_add_list = rd_list_new(1, rd_kafka_toppar_destroy_free); + } + + if(!rd_list_find(*toppars_add_list, rktp, rd_list_cmp_ptr)) + rd_list_add(*toppars_add_list, rd_kafka_toppar_keep(rktp)); + + /* Remove from removing toppars if present there. */ + if (*toppars_remove_list) { + rd_kafka_toppar_t *removed_rktp = rd_list_remove(*toppars_remove_list, rktp); + if(removed_rktp) { + rd_kafka_toppar_destroy(removed_rktp); + if(rd_list_empty(*toppars_remove_list)) { + rd_list_destroy(*toppars_remove_list); + *toppars_remove_list = NULL; + } + } + + } +} + +/** + * @brief Add description. + * + * In some scenarios, we don't have leader information present while assignment is done. In which case, + * when the leader is known later, we need to add the toppar to the broker's share fetch session. Being called from two places: + * 1) when a toppar is being added to the assignment in cgrp. + * 2) when a toppar is being added to the leader + * + * @locality broker thread + * @locks toppar lock + * @locks broker lock + */ +static void rd_kafka_broker_share_session_toppar_add(rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp) { + /** + * TODO KIP-932: + * * Check if rktp is present in current session already or not? + * * Check if rktp is already present in toppars_to_add? + */ + if (RD_KAFKA_IS_SHARE_CONSUMER(rktp->rktp_rkt->rkt_rk)) { + rd_kafka_broker_share_session_add_remove_toppar( + &rkb->rkb_share_fetch_session.toppars_to_add, &rkb->rkb_share_fetch_session.toppars_to_forget, rktp); + } +} + +/** + * @brief Add description. + * + * In some scenarios, we have to move the toppar out of the broker's share fetch session like leader migration to another broker. + * Being called from two places: + * 1) when a toppar is being removed from the assignment in cgrp. + * 2) when a toppar is being removed from the leader. + * + * @locality broker thread + * @locks toppar lock + * @locks broker lock + */ +static void rd_kafka_broker_share_session_toppar_remove(rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp) { + /** + * TODO KIP-932: + * * Check if rktp is present in current session already or not? No need to add if it is not present? + * * Check if rktp is already present in toppars_to_forget? + */ + if (RD_KAFKA_IS_SHARE_CONSUMER(rktp->rktp_rkt->rkt_rk)) { + rd_kafka_broker_share_session_add_remove_toppar( + &rkb->rkb_share_fetch_session.toppars_to_forget, &rkb->rkb_share_fetch_session.toppars_to_add, rktp); + } +} + /** * @brief Serve a broker op (an op posted by another thread to be handled by * this broker's thread). @@ -3323,6 +3403,9 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) { rd_kafka_broker_lock(rkb); TAILQ_INSERT_TAIL(&rkb->rkb_toppars, rktp, rktp_rkblink); rkb->rkb_toppar_cnt++; + if(rd_kafka_toppar_is_on_cgrp(rktp, rd_false)) { + rd_kafka_broker_share_session_toppar_add(rkb, rktp); + } rd_kafka_broker_unlock(rkb); rktp->rktp_broker = rkb; rd_assert(!rktp->rktp_msgq_wakeup_q); @@ -3421,6 +3504,7 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) { rd_kafka_broker_lock(rkb); TAILQ_REMOVE(&rkb->rkb_toppars, rktp, rktp_rkblink); rkb->rkb_toppar_cnt--; + rd_kafka_broker_share_session_toppar_remove(rkb, rktp); rd_kafka_broker_unlock(rkb); rd_kafka_broker_destroy(rktp->rktp_broker); if (rktp->rktp_msgq_wakeup_q) { @@ -3467,6 +3551,70 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) { rd_kafka_brokers_broadcast_state_change(rkb->rkb_rk); break; + case RD_KAFKA_OP_SHARE_FETCH: + rd_rkb_dbg(rkb, BROKER, "SHAREFETCH", + "Received SHARE_FETCH op for broker %s with " + "should_fetch = %d, should_leave = %d", + rd_kafka_broker_name(rkb), + rko->rko_u.share_fetch.should_fetch, + rko->rko_u.share_fetch.should_leave); + /* This is only temporary handling for testing to avoid crashing + * on assert - the code below will automatically enqueue a + * reply which is not the final behaviour. */ + /* Insert errors randomly for testing, remove this code once + * actual errors can be tested via the mock broker. */ + // if (rd_jitter(0, 10) > 7) { + // rd_rkb_dbg(rkb, CGRP, "SHAREFETCH", + // "Injecting error! %s : %d", + // rd_kafka_broker_name(rkb), + // rko->rko_u.share_fetch.should_fetch); + + // rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR__STATE); + // rko = NULL; + // } + + if(rko->rko_u.share_fetch.should_leave) { + rd_kafka_dbg(rkb->rkb_rk, BROKER, "SHAREFETCH", + "Processing SHARE_FETCH op: " + "should_leave is true"); + rd_kafka_broker_share_fetch_leave(rkb, rko, rd_clock()); + rko = NULL; /* the rko is reused for the reply */ + break; + } + + if (rd_kafka_broker_or_instance_terminating(rkb)) { + rd_kafka_dbg(rkb->rkb_rk, BROKER, "SHAREFETCH", + "Ignoring SHARE_FETCH op: " + "instance or broker is terminating"); + rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR__DESTROY); + } else if(rkb->rkb_fetching) { + rd_kafka_dbg(rkb->rkb_rk, BROKER, "SHAREFETCH", + "Ignoring SHARE_FETCH op: " + "already fetching"); + rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS); + } + + rd_kafka_broker_share_fetch(rkb, rko, rd_clock()); + + // if (!rko->rko_u.share_fetch.should_fetch) { + // rd_kafka_dbg(rkb->rkb_rk, BROKER, "SHAREFETCH", + // "Ignoring SHARE_FETCH op: " + // "should_fetch is false"); + // rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR__NOOP); + // break; + // } + + // if(rkb->rkb_state != RD_KAFKA_BROKER_STATE_UP) { + // rd_kafka_dbg(rkb->rkb_rk, BROKER, "SHAREFETCH", + // "Connection not up: Sending connect in progress as reply"); + // rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR__STATE); + // break; + // } + + rko = NULL; /* the rko is reused for the reply */ + + break; + case RD_KAFKA_OP_TERMINATE: /* nop: just a wake-up. */ rd_rkb_dbg(rkb, BROKER, "TERM", @@ -3552,6 +3700,28 @@ rd_kafka_broker_op_serve(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) { wakeup = rd_true; break; + + case RD_KAFKA_OP_SHARE_SESSION_PARTITION_ADD: + rd_rkb_dbg(rkb, CGRP, "SHARESESSION", + "Received SHARE_SESSION_PARTITION_ADD op for " + "topic %s [%" PRId32 "]", + rko->rko_rktp->rktp_rkt->rkt_topic->str, + rko->rko_rktp->rktp_partition); + + rd_kafka_broker_share_session_toppar_add( + rkb, rko->rko_rktp); + break; + + case RD_KAFKA_OP_SHARE_SESSION_PARTITION_REMOVE: + rd_rkb_dbg(rkb, CGRP, "SHARESESSION", + "Received SHARE_SESSION_PARTITION_REMOVE op for " + "topic %s [%" PRId32 "]", + rko->rko_rktp->rktp_rkt->rkt_topic->str, + rko->rko_rktp->rktp_partition); + + rd_kafka_broker_share_session_toppar_remove( + rkb, rko->rko_rktp); + break; default: rd_kafka_assert(rkb->rkb_rk, !*"unhandled op type"); @@ -4272,6 +4442,65 @@ static void rd_kafka_broker_producer_serve(rd_kafka_broker_t *rkb, } +// void rd_kafka_broker_update_share_fetch_session(rd_kafka_broker_t *rkb) { +// rd_kafka_toppar_t *rktp, *rktp_tmp; +// rd_bool_t needs_update = rd_false; + +// TAILQ_FOREACH(rktp, &rkb->rkb_share_fetch_session.toppars_in_session, rktp_rkb_session_link) { +// rd_kafka_toppar_is_valid_to_send_for_share_fetch(rktp); +// } + +// if (needs_update) +// rd_kafka_toppar_share_fetch_session_update(rkb); +// } + + +/** + * Consumer serving + * + * TODO KIP-932: Fix timeouts. + */ +static void rd_kafka_broker_share_consumer_serve(rd_kafka_broker_t *rkb, + rd_ts_t abs_timeout) { + unsigned int initial_state = rkb->rkb_state; + rd_ts_t now; + + rd_kafka_assert(rkb->rkb_rk, thrd_is_current(rkb->rkb_thread)); + + rd_kafka_broker_lock(rkb); + + while (!rd_kafka_broker_terminating(rkb) && + rkb->rkb_state == initial_state && + abs_timeout > (now = rd_clock())) { + rd_ts_t min_backoff = abs_timeout; + + rd_kafka_broker_unlock(rkb); + + if (rkb->rkb_toppar_cnt > 0 && + rkb->rkb_share_fetch_session.epoch >= 0 && + rkb->rkb_state != RD_KAFKA_BROKER_STATE_UP) { + /* There are partitions to fetch but the + * connection is not up. */ + rkb->rkb_persistconn.internal++; + } + + /* Check and move retry buffers */ + if (unlikely(rd_atomic32_get(&rkb->rkb_retrybufs.rkbq_cnt) > 0)) + rd_kafka_broker_retry_bufs_move(rkb, &min_backoff); + + if (min_backoff > abs_timeout) + min_backoff = abs_timeout; + + // rd_kafka_broker_update_share_fetch_session(rkb); + + if (rd_kafka_broker_ops_io_serve(rkb, min_backoff)) + return; /* Wakeup */ + + rd_kafka_broker_lock(rkb); + } + + rd_kafka_broker_unlock(rkb); +} /** * Consumer serving @@ -4482,6 +4711,8 @@ static void rd_kafka_broker_serve(rd_kafka_broker_t *rkb, int timeout_ms) { if (rkb->rkb_rk->rk_type == RD_KAFKA_PRODUCER) rd_kafka_broker_producer_serve(rkb, abs_timeout); + else if (RD_KAFKA_IS_SHARE_CONSUMER(rkb->rkb_rk)) + rd_kafka_broker_share_consumer_serve(rkb, abs_timeout); else if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER) rd_kafka_broker_consumer_serve(rkb, abs_timeout); @@ -4712,6 +4943,40 @@ static int rd_kafka_broker_thread_main(void *arg) { (int)rd_kafka_bufq_cnt(&rkb->rkb_outbufs), (int)rd_kafka_bufq_cnt(&rkb->rkb_waitresps), (int)rd_kafka_bufq_cnt(&rkb->rkb_retrybufs), r); + + rd_rkb_dbg(rkb, BROKER, "TERMINATE", + "Partitions in fetch session: %d", + rkb->rkb_share_fetch_session.toppars_in_session_cnt); + if(rkb->rkb_share_fetch_session.toppars_to_add) + rd_rkb_dbg(rkb, BROKER, "TERMINATE", + "Partitions to add to fetch session: %d", + rd_list_cnt( + rkb->rkb_share_fetch_session.toppars_to_add)); + if(rkb->rkb_share_fetch_session.toppars_to_forget) { + rd_rkb_dbg(rkb, BROKER, "TERMINATE", + "Partitions to forget from fetch session: %d", + rd_list_cnt( + rkb->rkb_share_fetch_session.toppars_to_forget)); + rd_kafka_toppar_t *rktp; + int i; + RD_LIST_FOREACH(rktp, rkb->rkb_share_fetch_session.toppars_to_forget, i) { + rd_rkb_dbg(rkb, BROKER, "TERMINATE", + " - %.*s [%" PRId32 "]", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + } + + } + if(rkb->rkb_share_fetch_session.adding_toppars) + rd_rkb_dbg(rkb, BROKER, "TERMINATE", + "Partitions being added to fetch session: %d", + rd_list_cnt( + rkb->rkb_share_fetch_session.adding_toppars)); + if(rkb->rkb_share_fetch_session.forgetting_toppars) + rd_rkb_dbg(rkb, BROKER, "TERMINATE", + "Partitions being forgotten from fetch session: %d", + rd_list_cnt( + rkb->rkb_share_fetch_session.forgetting_toppars)); } } @@ -4794,6 +5059,9 @@ void rd_kafka_broker_destroy_final(rd_kafka_broker_t *rkb) { rd_assert(TAILQ_EMPTY(&rkb->rkb_waitresps.rkbq_bufs)); rd_assert(TAILQ_EMPTY(&rkb->rkb_retrybufs.rkbq_bufs)); rd_assert(TAILQ_EMPTY(&rkb->rkb_toppars)); + rd_assert(TAILQ_EMPTY(&rkb->rkb_share_fetch_session.toppars_in_session)); + rd_assert(!rkb->rkb_share_fetch_session.toppars_to_add); + rd_assert(!rkb->rkb_share_fetch_session.toppars_to_forget); if (rkb->rkb_source != RD_KAFKA_INTERNAL && (rkb->rkb_rk->rk_conf.security_protocol == @@ -4920,11 +5188,16 @@ rd_kafka_broker_t *rd_kafka_broker_add(rd_kafka_t *rk, rkb->rkb_port = port; rkb->rkb_origname = rd_strdup(name); rkb->rkb_c.connections_max_idle_ms = -1; + rkb->rkb_share_fetch_session.epoch = 0; mtx_init(&rkb->rkb_lock, mtx_plain); mtx_init(&rkb->rkb_logname_lock, mtx_plain); rkb->rkb_logname = rd_strdup(rkb->rkb_name); TAILQ_INIT(&rkb->rkb_toppars); + TAILQ_INIT(&rkb->rkb_share_fetch_session.toppars_in_session); + rkb->rkb_share_fetch_session.toppars_in_session_cnt = 0; + rkb->rkb_share_fetch_session.toppars_to_forget = NULL; + rkb->rkb_share_fetch_session.toppars_to_add = NULL; CIRCLEQ_INIT(&rkb->rkb_active_toppars); TAILQ_INIT(&rkb->rkb_monitors); rd_kafka_bufq_init(&rkb->rkb_outbufs); @@ -6191,6 +6464,22 @@ void rd_kafka_broker_decommission(rd_kafka_t *rk, if (rd_atomic32_get(&rkb->termination_in_progress) > 0) return; + if(RD_KAFKA_IS_SHARE_CONSUMER(rk) && rkb->rkb_source == RD_KAFKA_LEARNED) { + rd_kafka_op_t *rko_sf; + rko_sf = rd_kafka_op_new(RD_KAFKA_OP_SHARE_FETCH); + rko_sf->rko_u.share_fetch.should_leave = rd_true; + rko_sf->rko_u.share_fetch.abs_timeout = 0; // TODO KIP-932: Check timeout part. + rko_sf->rko_u.share_fetch.should_fetch = rd_false; + rd_kafka_broker_keep(rkb); + rko_sf->rko_u.share_fetch.target_broker = rkb; + rko_sf->rko_replyq = RD_KAFKA_REPLYQ(rk->rk_ops, 0); + + rd_kafka_dbg(rk, BROKER, "SHAREFETCH", + "Enqueuing leave share fetch op on broker %s: decommissioning broker.", + rd_kafka_broker_name(rkb)); + rd_kafka_q_enq(rkb->rkb_ops, rko_sf); + } + rd_atomic32_add(&rkb->termination_in_progress, 1); /* Add broker's thread to wait_thrds list for later joining */ diff --git a/src/rdkafka_broker.h b/src/rdkafka_broker.h index a649b7445e..d6df843635 100644 --- a/src/rdkafka_broker.h +++ b/src/rdkafka_broker.h @@ -106,6 +106,34 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */ /* Toppars handled by this broker */ TAILQ_HEAD(, rd_kafka_toppar_s) rkb_toppars; + + struct { + TAILQ_HEAD(, rd_kafka_toppar_s) toppars_in_session; /* List of toppars + in the current + fetch session. + Any new added toppar in rkb_toppars will be added here after successful share fetch request. + Any removed toppar from rkb_toppars will be removed from here after successful share fetch request. + rkb_share_fetch_session.forgotten_toppars is calculated by rkb_share_fetch_session.toppars - rkb_toppars */ + int toppars_in_session_cnt; + rd_list_t *toppars_to_add; /* TODO KIP-932: Move this from `rd_list_t` to `TAILQ_HEAD(, rd_kafka_toppar_s)` for performance improvements. + * List of toppars that are added to rkb_toppars but not yet added to fetch session. + * Will be sent in next fetch request. + * Cleared when fetch session is reset or when fetch request is successful. */ + rd_list_t *adding_toppars; + rd_list_t *toppars_to_forget; /* TODO KIP-932: Move this from `rd_list_t` to `TAILQ_HEAD(, rd_kafka_toppar_s)` for performance improvements. + * List of toppars + * that are removed from rkb_toppars and sent in fetch request but not yet removed from fetch session. + * Cleared when fetch session is reset or when fetch request is successful. */ + rd_list_t *forgetting_toppars; + int32_t epoch; /* Current fetch session + * epoch, or -1 if leaving the session + * TODO KIP-932: Handle 0 and -1 properly. + * * Can we move from -1 to 0? + * * Maybe in some error case? + * * Is there a way in which we close a previous session and start a new one? + */ + } rkb_share_fetch_session; + int rkb_toppar_cnt; /* Active toppars that are eligible for: @@ -387,6 +415,12 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */ /** > 0 if this broker thread is terminating */ rd_atomic32_t termination_in_progress; + + /** + * Whether a share fetch should_fetch set is enqueued on + * this broker's op queue or not. + */ + rd_bool_t rkb_share_fetch_enqueued; }; #define rd_kafka_broker_keep(rkb) rd_refcnt_add(&(rkb)->rkb_refcnt) diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index d6348ba52d..7a4732e0a3 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -1086,7 +1086,91 @@ rd_kafka_cgrp_handle_ConsumerGroupHeartbeat_leave(rd_kafka_t *rk, goto err; } +static void rd_kafka_cgrp_handle_ShareGroupHeartbeat_leave( + rd_kafka_t *rk, + rd_kafka_broker_t *rkb, + rd_kafka_resp_err_t err, + rd_kafka_buf_t *rkbuf, + rd_kafka_buf_t *request, + void *opaque) { + rd_kafka_cgrp_t *rkcg = opaque; + const int log_decode_errors = LOG_ERR; + int16_t ErrorCode = 0; + + if (err) { + ErrorCode = err; + goto err; + } + + rd_kafka_buf_read_throttle_time(rkbuf); + + rd_kafka_buf_read_i16(rkbuf, &ErrorCode); +err: + if (ErrorCode) + rd_kafka_dbg( + rkb->rkb_rk, CGRP, "LEAVEGROUP", + "ShareGroupHeartbeat response error in state %s: %s", + rd_kafka_cgrp_state_names[rkcg->rkcg_state], + rd_kafka_err2str(ErrorCode)); + else + rd_kafka_dbg( + rkb->rkb_rk, CGRP, "LEAVEGROUP", + "ShareGroupHeartbeat response received in state %s", + rd_kafka_cgrp_state_names[rkcg->rkcg_state]); + rd_kafka_cgrp_consumer_reset(rkcg); + if (ErrorCode != RD_KAFKA_RESP_ERR__DESTROY) { + rd_assert(thrd_is_current(rk->rk_thread)); + rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_WAIT_LEAVE; + rd_kafka_cgrp_try_terminate(rkcg); + } + return; +err_parse: + ErrorCode = rkbuf->rkbuf_err; + goto err; +} + +static void rd_kafka_cgrp_share_consumer_leave(rd_kafka_cgrp_t *rkcg) { + int32_t member_epoch = -1; + + if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WAIT_LEAVE) { + rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "LEAVE", + "Group \"%.*s\": leave (in state %s): " + "ShareGroupHeartbeat already in-transit", + RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), + rd_kafka_cgrp_state_names[rkcg->rkcg_state]); + return; + } + + rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "LEAVE", + "Group \"%.*s\": leave (in state %s)", + RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), + rd_kafka_cgrp_state_names[rkcg->rkcg_state]); + + rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_WAIT_LEAVE; + + if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_UP) { + rd_rkb_dbg(rkcg->rkcg_curr_coord, CONSUMER, "LEAVE", + "Share consumer: leaving group"); + rd_kafka_ShareGroupHeartbeatRequest( + rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_member_id, + member_epoch, + NULL /* no rack */, + NULL /* no subscription topics */, + RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0), + rd_kafka_cgrp_handle_ShareGroupHeartbeat_leave, rkcg); + } else { + rd_kafka_cgrp_handle_ShareGroupHeartbeat_leave( + rkcg->rkcg_rk, rkcg->rkcg_coord, + RD_KAFKA_RESP_ERR__WAIT_COORD, NULL, NULL, rkcg); + } +} + static void rd_kafka_cgrp_consumer_leave(rd_kafka_cgrp_t *rkcg) { + if (RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk)) { + rd_kafka_cgrp_share_consumer_leave(rkcg); + return; + } + int32_t member_epoch = -1; if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WAIT_LEAVE) { @@ -3369,6 +3453,304 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk, } } +void rd_kafka_cgrp_handle_ShareGroupHeartbeat(rd_kafka_t *rk, + rd_kafka_broker_t *rkb, + rd_kafka_resp_err_t err, + rd_kafka_buf_t *rkbuf, + rd_kafka_buf_t *request, + void *opaque) { + rd_kafka_cgrp_t *rkcg = rk->rk_cgrp; + const int log_decode_errors = LOG_ERR; + int16_t error_code = 0; + int actions = 0; + rd_kafkap_str_t error_str = RD_KAFKAP_STR_INITIALIZER_EMPTY; + rd_kafkap_str_t member_id; + int32_t member_epoch; + int32_t heartbeat_interval_ms; + int8_t are_assignments_present; + + if (err == RD_KAFKA_RESP_ERR__DESTROY) + return; + + rd_dassert(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT); + + if (rd_kafka_cgrp_will_leave(rkcg)) + err = RD_KAFKA_RESP_ERR__OUTDATED; + if (err) + goto err; + + rd_kafka_buf_read_throttle_time(rkbuf); + rd_kafka_buf_read_i16(rkbuf, &error_code); + rd_kafka_buf_read_str(rkbuf, &error_str); + + if (error_code) { + err = error_code; + goto err; + } + + rd_kafka_buf_read_str(rkbuf, &member_id); + if (!RD_KAFKAP_STR_IS_NULL(&member_id)) { + rd_kafka_cgrp_set_member_id(rkcg, member_id.str); + } + + rd_kafka_buf_read_i32(rkbuf, &member_epoch); + rkcg->rkcg_generation_id = member_epoch; + + rd_kafka_buf_read_i32(rkbuf, &heartbeat_interval_ms); + if (heartbeat_interval_ms > 0) { + rkcg->rkcg_heartbeat_intvl_ms = heartbeat_interval_ms; + } + + rd_kafka_buf_read_i8(rkbuf, &are_assignments_present); + + if (are_assignments_present == 1) { + rd_kafka_topic_partition_list_t *assigned_topic_partitions; + const rd_kafka_topic_partition_field_t assignments_fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + assigned_topic_partitions = rd_kafka_buf_read_topic_partitions( + rkbuf, rd_true, rd_false /* Don't use Topic Name */, 0, + assignments_fields); + + rd_kafka_dbg( + rk, CGRP, "HEARTBEAT", + "ShareGroupHeartbeat response received " + "assigned_topic_partitions size %d", + assigned_topic_partitions->cnt); + + if (rd_kafka_is_dbg(rk, CGRP)) { + char assigned_topic_partitions_str[512] = "NULL"; + + if (assigned_topic_partitions) { + rd_kafka_topic_partition_list_str( + assigned_topic_partitions, + assigned_topic_partitions_str, + sizeof(assigned_topic_partitions_str), 0); + } + + rd_kafka_dbg( + rk, CGRP, "HEARTBEAT", + "ShareGroupHeartbeat response received target " + "assignment \"%s\"", + assigned_topic_partitions_str); + } + + if (assigned_topic_partitions) { + RD_IF_FREE(rkcg->rkcg_next_target_assignment, + rd_kafka_topic_partition_list_destroy); + rkcg->rkcg_next_target_assignment = NULL; + if (rd_kafka_cgrp_consumer_is_new_assignment_different( + rkcg, assigned_topic_partitions)) { + rkcg->rkcg_next_target_assignment = + assigned_topic_partitions; + } else { + rd_kafka_topic_partition_list_destroy( + assigned_topic_partitions); + assigned_topic_partitions = NULL; + } + } + } + + if (rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY && + (rkcg->rkcg_consumer_flags & RD_KAFKA_CGRP_CONSUMER_F_WAIT_ACK) && + rkcg->rkcg_target_assignment) { + if (rkcg->rkcg_consumer_flags & + RD_KAFKA_CGRP_CONSUMER_F_SENDING_ACK) { + if (rkcg->rkcg_current_assignment) + rd_kafka_topic_partition_list_destroy( + rkcg->rkcg_current_assignment); + rkcg->rkcg_current_assignment = + rd_kafka_topic_partition_list_copy( + rkcg->rkcg_target_assignment); + rd_kafka_topic_partition_list_destroy( + rkcg->rkcg_target_assignment); + rkcg->rkcg_target_assignment = NULL; + rkcg->rkcg_consumer_flags &= + ~RD_KAFKA_CGRP_CONSUMER_F_WAIT_ACK; + + if (rd_kafka_is_dbg(rkcg->rkcg_rk, CGRP)) { + char rkcg_current_assignment_str[512] = "NULL"; + + rd_kafka_topic_partition_list_str( + rkcg->rkcg_current_assignment, + rkcg_current_assignment_str, + sizeof(rkcg_current_assignment_str), 0); + + rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "HEARTBEAT", + "Target assignment acked, new " + "current assignment " + " \"%s\"", + rkcg_current_assignment_str); + } + } else if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION) { + /* We've finished reconciliation but we weren't + * sending an ack, need to send a new HB with the ack. + */ + rd_kafka_cgrp_consumer_expedite_next_heartbeat( + rkcg, "not subscribed anymore"); + } + } + + + if (rkcg->rkcg_consumer_flags & + RD_KAFKA_CGRP_CONSUMER_F_SERVE_PENDING && + rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY) { + /* TODO: Check if this should be done only for the + * steady state? + */ + rd_kafka_assignment_serve(rk); + rkcg->rkcg_consumer_flags &= + ~RD_KAFKA_CGRP_CONSUMER_F_SERVE_PENDING; + } + + if (rkcg->rkcg_next_target_assignment) { + if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION) { + rd_kafka_cgrp_consumer_next_target_assignment_request_metadata( + rk, rkb); + } else { + /* Consumer left the group sending an HB request + * while this one was in-flight. */ + rd_kafka_topic_partition_list_destroy( + rkcg->rkcg_next_target_assignment); + rkcg->rkcg_next_target_assignment = NULL; + } + } + + if (rd_kafka_cgrp_consumer_subscription_preconditions_met(rkcg)) + rd_kafka_cgrp_consumer_expedite_next_heartbeat( + rkcg, "send new subscription"); + + rkcg->rkcg_consumer_flags &= + ~RD_KAFKA_CGRP_CONSUMER_F_SENDING_NEW_SUBSCRIPTION & + ~RD_KAFKA_CGRP_CONSUMER_F_SEND_FULL_REQUEST; + rd_kafka_cgrp_maybe_clear_heartbeat_failed_err(rkcg); + rkcg->rkcg_last_heartbeat_err = RD_KAFKA_RESP_ERR_NO_ERROR; + rkcg->rkcg_expedite_heartbeat_retries = 0; + rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT; + + return; + +err_parse: + err = rkbuf->rkbuf_err; +err: + rkcg->rkcg_last_heartbeat_err = err; + rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT; + switch (err) { + case RD_KAFKA_RESP_ERR__DESTROY: + /* quick cleanup */ + return; + + case RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS: + rd_kafka_dbg( + rkcg->rkcg_rk, CONSUMER, "HEARTBEAT", + "ShareGroupHeartbeat failed due to coordinator (%s) " + "loading in progress: %s: " + "retrying", + rkcg->rkcg_curr_coord + ? rd_kafka_broker_name(rkcg->rkcg_curr_coord) + : "none", + rd_kafka_err2str(err)); + actions = RD_KAFKA_ERR_ACTION_RETRY; + break; + + case RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP: + case RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR__TRANSPORT: + rd_kafka_dbg( + rkcg->rkcg_rk, CONSUMER, "HEARTBEAT", + "ShareGroupHeartbeat failed due to coordinator (%s) " + "no longer available: %s: " + "re-querying for coordinator", + rkcg->rkcg_curr_coord + ? rd_kafka_broker_name(rkcg->rkcg_curr_coord) + : "none", + rd_kafka_err2str(err)); + /* Remain in joined state and keep querying for coordinator */ + actions = RD_KAFKA_ERR_ACTION_REFRESH; + break; + + case RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID: + rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER, "HEARTBEAT", + "ShareGroupHeartbeat failed due to: %s: " + "will rejoin the group", + rd_kafka_err2str(err)); + rkcg->rkcg_consumer_flags |= + RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN; + return; + + case RD_KAFKA_RESP_ERR_INVALID_REQUEST: + case RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED: + case RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION: + case RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE: + case RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED: + actions = RD_KAFKA_ERR_ACTION_FATAL; + break; + + default: + actions = rd_kafka_err_action( + rkb, err, request, + + RD_KAFKA_ERR_ACTION_SPECIAL, + RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED, + + RD_KAFKA_ERR_ACTION_END); + break; + } + + if (actions & RD_KAFKA_ERR_ACTION_FATAL) { + rd_kafka_set_fatal_error( + rkcg->rkcg_rk, err, + "ShareGroupHeartbeat fatal error: %s", + rd_kafka_err2str(err)); + rd_kafka_cgrp_revoke_all_rejoin_maybe( + rkcg, rd_true, /*assignments lost*/ + rd_true, /*initiating*/ + "Fatal error in ShareGroupHeartbeat API response"); + return; + } + + if (!rkcg->rkcg_heartbeat_intvl_ms) { + /* When an error happens on first HB, it should be always + * retried, unless fatal, to avoid entering a tight loop + * and to use exponential backoff. */ + actions |= RD_KAFKA_ERR_ACTION_RETRY; + } + + if (actions & RD_KAFKA_ERR_ACTION_REFRESH) { + /* Re-query for coordinator */ + rkcg->rkcg_consumer_flags |= + RD_KAFKA_CGRP_CONSUMER_F_SEND_FULL_REQUEST; + rd_kafka_cgrp_coord_query(rkcg, rd_kafka_err2str(err)); + /* If coordinator changes, HB will be expedited. */ + } + + if (actions & RD_KAFKA_ERR_ACTION_SPECIAL) { + rd_ts_t min_error_interval = + RD_MAX(rkcg->rkcg_heartbeat_intvl_ms * 1000, + /* default group.consumer.heartbeat.interval.ms */ + 5000000); + if (rkcg->rkcg_last_err != err || + (rd_clock() > + rkcg->rkcg_ts_last_err + min_error_interval)) { + rd_kafka_cgrp_set_last_err(rkcg, err); + rd_kafka_consumer_err( + rkcg->rkcg_q, rd_kafka_broker_id(rkb), err, 0, NULL, + NULL, err, + "ShareGroupHeartbeat failed: %s%s%.*s", + rd_kafka_err2str(err), + RD_KAFKAP_STR_LEN(&error_str) ? ": " : "", + RD_KAFKAP_STR_PR(&error_str)); + } + } + + if (actions & RD_KAFKA_ERR_ACTION_RETRY && + rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION && + !rd_kafka_cgrp_will_leave(rkcg) && + rd_kafka_buf_retry(rkb, request)) { + /* Retry */ + rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT; + } +} /** * @brief Handle Heartbeat response. @@ -3679,6 +4061,19 @@ static void rd_kafka_cgrp_partition_add(rd_kafka_cgrp_t *rkcg, rd_kafka_toppar_lock(rktp); rd_assert(!(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_CGRP)); rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_ON_CGRP; + if(RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk) && rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_RKB) { + rd_kafka_op_t *rko; + rko = rd_kafka_op_new(RD_KAFKA_OP_SHARE_SESSION_PARTITION_ADD); + rko->rko_rktp = rd_kafka_toppar_keep(rktp); /* refcnt from _add op */ + rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "SHARESESSPARTCGRPADD", + "Group \"%s\": enqueue partition add for %s [%" PRId32 "] " + "on broker %s", + rkcg->rkcg_group_id->str, + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition, + rd_kafka_broker_name(rktp->rktp_broker)); + rd_kafka_q_enq(rktp->rktp_broker->rkb_ops, rko); + } rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_keep(rktp); @@ -3701,6 +4096,20 @@ static void rd_kafka_cgrp_partition_del(rd_kafka_cgrp_t *rkcg, rd_assert(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_CGRP); rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_ON_CGRP; + if(RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk) && rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_RKB) { + rd_kafka_op_t *rko; + rko = rd_kafka_op_new(RD_KAFKA_OP_SHARE_SESSION_PARTITION_REMOVE); + rko->rko_rktp = rd_kafka_toppar_keep(rktp); /* refcnt from _add op */ + rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "SHARESESSPARTCGRPDEL", + "Group \"%s\": enqueue partition remove for %s [%" PRId32 "] " + "on broker %s", + rkcg->rkcg_group_id->str, + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition, + rd_kafka_broker_name(rktp->rktp_broker)); + rd_kafka_q_enq(rktp->rktp_broker->rkb_ops, rko); + } + rd_kafka_toppar_purge_internal_fetch_queue_maybe(rktp); rd_kafka_toppar_unlock(rktp); @@ -6150,6 +6559,14 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg, } rkcg->rkcg_expedite_heartbeat_retries++; + + if (RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk)) { + rd_kafka_ShareGroupHeartbeatRequest(rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_member_id, + member_epoch, rkcg_client_rack, rkcg_subscription_topics, RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0), + rd_kafka_cgrp_handle_ShareGroupHeartbeat, NULL); + return; + } + rd_kafka_ConsumerGroupHeartbeatRequest( rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_member_id, member_epoch, rkcg_group_instance_id, rkcg_client_rack, @@ -6211,6 +6628,15 @@ void rd_kafka_cgrp_consumer_serve(rd_kafka_cgrp_t *rkcg) { "member fenced - rejoining"); } + /* There should be no fencing, hence no rejoining - these asserts are to test only, we don't actually need them. */ + rd_dassert(!(RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk) && + (rkcg->rkcg_consumer_flags & + RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN))); + rd_dassert(!(RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk) && + (rkcg->rkcg_consumer_flags & + RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN_TO_COMPLETE))); + + switch (rkcg->rkcg_join_state) { case RD_KAFKA_CGRP_JOIN_STATE_INIT: rkcg->rkcg_consumer_flags &= @@ -6300,6 +6726,7 @@ rd_kafka_cgrp_consumer_subscribe(rd_kafka_cgrp_t *rkcg, /* If member is leaving, new subscription * will be applied after the leave * ConsumerGroupHeartbeat */ + /* MILIND: how is new subscription applied after heartbeat, check it. */ if (!rd_kafka_cgrp_will_leave(rkcg)) rd_kafka_cgrp_consumer_apply_next_subscribe(rkcg); } else { diff --git a/src/rdkafka_cgrp.h b/src/rdkafka_cgrp.h index 79a734f5fb..8f6afedc7d 100644 --- a/src/rdkafka_cgrp.h +++ b/src/rdkafka_cgrp.h @@ -377,6 +377,14 @@ typedef struct rd_kafka_cgrp_s { /* Timestamp of last rebalance start */ rd_ts_t rkcg_ts_rebalance_start; + struct { + size_t last_partition_picked; /* For round-robin + * partition picking */ + rd_kafka_timer_t share_fetch_fanout_tmr; /**< Timer for + * share fetch + * fanout */ + + } rkcg_share; } rd_kafka_cgrp_t; diff --git a/src/rdkafka_conf.c b/src/rdkafka_conf.c index 1f8bbf106b..fe49950da8 100644 --- a/src/rdkafka_conf.c +++ b/src/rdkafka_conf.c @@ -1509,6 +1509,11 @@ static const struct rd_kafka_property rd_kafka_properties[] = { "which indicates where this client is physically located. It " "corresponds with the broker config `broker.rack`.", .sdef = ""}, + {_RK_GLOBAL | _RK_HIDDEN, "share.consumer", _RK_C_BOOL, + _RK(share.is_share_consumer), "tba description", 0, 1, 0}, + {_RK_GLOBAL | _RK_CONSUMER, "max.poll.records", _RK_C_INT, + _RK(share.max_poll_records), "tba description,", 1, INT_MAX, 500}, + /* Global producer properties */ {_RK_GLOBAL | _RK_PRODUCER | _RK_HIGH, "transactional.id", _RK_C_STR, diff --git a/src/rdkafka_conf.h b/src/rdkafka_conf.h index 92e5193eb7..ea9e216efa 100644 --- a/src/rdkafka_conf.h +++ b/src/rdkafka_conf.h @@ -470,6 +470,11 @@ struct rd_kafka_conf_s { rd_kafkap_str_t *client_rack; + struct { + int is_share_consumer; /**< Is this a share consumer? */ + int max_poll_records; /**< Max records returned per poll */ + } share; + /* * Producer configuration */ diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index e275ee5a88..d56cc34056 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -885,6 +885,492 @@ rd_kafka_fetch_reply_handle(rd_kafka_broker_t *rkb, } +static rd_kafka_resp_err_t rd_kafka_share_fetch_reply_handle_partition( + rd_kafka_broker_t *rkb, + const rd_kafkap_str_t *topic, + rd_kafka_topic_t *rkt /*possibly NULL*/, + rd_kafka_buf_t *rkbuf, + rd_kafka_buf_t *request) { + + /* TODO: KIP-932: Check rd_kafka_fetch_reply_handle_partition + * and modify as needed for ShareFetch. + */ + int32_t PartitionId; + int16_t PartitionFetchErrorCode; + rd_kafkap_str_t PartitionFetchErrorStr = RD_KAFKAP_STR_INITIALIZER_EMPTY; + int16_t AcknowledgementErrorCode; + rd_kafkap_str_t AcknowledgementErrorStr = RD_KAFKAP_STR_INITIALIZER_EMPTY; + rd_kafkap_CurrentLeader_t CurrentLeader; + int32_t MessageSetSize; + rd_kafka_toppar_t *rktp = NULL; + struct rd_kafka_toppar_ver tver; + rd_slice_t save_slice; + const int log_decode_errors = LOG_ERR; + rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; + int32_t AcquiredRecordsArrayCnt; + int64_t FirstOffset; + int64_t LastOffset; + int16_t DeliveryCount; + int i; + + rd_kafka_buf_read_i32(rkbuf, &PartitionId); // Partition + rd_kafka_buf_read_i16(rkbuf, &PartitionFetchErrorCode); // PartitionFetchError + rd_kafka_buf_read_str(rkbuf, &PartitionFetchErrorStr); // ErrorString + /* TODO KIP-932: We should reset (to INVALID) previous acknowledgement information in the reply + or maybe while sending the request itself? */ + rd_kafka_buf_read_i16(rkbuf, &AcknowledgementErrorCode); // AcknowledgementError + rd_kafka_buf_read_str(rkbuf, &AcknowledgementErrorStr); // AcknowledgementErrorString + rd_kafka_buf_read_CurrentLeader(rkbuf, &CurrentLeader); // CurrentLeader + + /* Compact Records Array */ + rd_kafka_buf_read_arraycnt(rkbuf, &MessageSetSize, -1); + + if (unlikely(MessageSetSize < 0)) + rd_kafka_buf_parse_fail( + rkbuf, + "%.*s [%" PRId32 "]: invalid MessageSetSize %" PRId32, + RD_KAFKAP_STR_PR(topic), PartitionId, MessageSetSize); + + /* Look up topic+partition */ + if (likely(rkt != NULL)) { + rd_kafka_topic_rdlock(rkt); + rktp = rd_kafka_toppar_get(rkt, PartitionId, + 0 /*no ua-on-miss*/); + rd_kafka_topic_rdunlock(rkt); + } + + if (unlikely(!rkt || !rktp)) { + rd_rkb_dbg(rkb, TOPIC, "UNKTOPIC", + "Received Fetch response (error %hu) for unknown " + "topic %.*s [%" PRId32 "]: ignoring", + PartitionFetchErrorCode, RD_KAFKAP_STR_PR(topic), + PartitionId); + rd_kafka_buf_skip(rkbuf, MessageSetSize); + goto done; + } + + tver.rktp = rktp; + tver.version = rktp->rktp_fetch_version; + + + /* No error, clear any previous fetch error. */ + rktp->rktp_last_error = RD_KAFKA_RESP_ERR_NO_ERROR; + + if(MessageSetSize > 0) { + /** + * Parse MessageSet + */ + if (!rd_slice_narrow_relative(&rkbuf->rkbuf_reader, &save_slice, + (size_t) MessageSetSize)) + rd_kafka_buf_check_len(rkbuf, MessageSetSize); + + /* Parse messages + TODO KIP-932: This part might raise issue as We are adding messages + to the consumer queue in partition by partition manner. + The poll returns messages as soon as they are available in the queue, + so messages for different partitions in the same fetch request might + not be sent at once to the user. + */ + err = rd_kafka_msgset_parse(rkbuf, request, rktp, NULL, &tver); + + rd_slice_widen(&rkbuf->rkbuf_reader, &save_slice); + /* Continue with next partition regardless of + * parse errors (which are partition-specific) */ + + } + + rd_kafka_buf_read_arraycnt(rkbuf, &AcquiredRecordsArrayCnt, -1); // AcquiredRecordsArrayCnt + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", "%.*s [%" PRId32 "] : Share Acknowledgement Count: %ld, AcquiredRecordsArrayCnt: %d\n", + RD_KAFKAP_STR_PR(topic), PartitionId, + rktp->rktp_share_acknowledge_count, AcquiredRecordsArrayCnt); + rd_dassert(rktp->rktp_share_acknowledge_count == 0); + rd_dassert(rktp->rktp_share_acknowledge == NULL); + if(AcquiredRecordsArrayCnt > 0) { + rktp->rktp_share_acknowledge_count = AcquiredRecordsArrayCnt; + rktp->rktp_share_acknowledge = rd_calloc(AcquiredRecordsArrayCnt, + sizeof(*rktp->rktp_share_acknowledge)); + for (i = 0; i < AcquiredRecordsArrayCnt; i++) { + rd_kafka_buf_read_i64(rkbuf, &FirstOffset); // FirstOffset + rd_kafka_buf_read_i64(rkbuf, &LastOffset); // LastOffset + rd_kafka_buf_read_i16(rkbuf, &DeliveryCount); // DeliveryCount + rd_kafka_buf_skip_tags(rkbuf); // AcquiredRecords tags + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", + "%.*s [%" PRId32 "]: Acquired Records from offset %" PRId64 + " to %" PRId64 ", DeliveryCount %" PRId16, + RD_KAFKAP_STR_PR(topic), PartitionId, + FirstOffset, LastOffset, DeliveryCount); + rktp->rktp_share_acknowledge[i].first_offset = FirstOffset; + rktp->rktp_share_acknowledge[i].last_offset = LastOffset; + rktp->rktp_share_acknowledge[i].delivery_count = DeliveryCount; + } + } + + rd_kafka_buf_skip_tags(rkbuf); // Partition tags + + goto done; + +err_parse: + if (rktp) + rd_kafka_toppar_destroy(rktp); /*from get()*/ + return rkbuf->rkbuf_err; + +done: + if (likely(rktp != NULL)) + rd_kafka_toppar_destroy(rktp); /*from get()*/ + + return err; + + } + + +/** + * Parses and handles a ShareFetch reply. + * Returns 0 on success or an error code on failure. + * + * TODO KIP-932: Change return type to proper error with message. See `rd_kafka_error_t *`. + */ +static rd_kafka_resp_err_t +rd_kafka_share_fetch_reply_handle(rd_kafka_broker_t *rkb, + rd_kafka_buf_t *rkbuf, + rd_kafka_buf_t *request) { + int32_t TopicArrayCnt; + int i; + const int log_decode_errors = LOG_ERR; + rd_kafka_topic_t *rkt = NULL; + int16_t ErrorCode = RD_KAFKA_RESP_ERR_NO_ERROR; + rd_kafkap_str_t ErrorStr = RD_KAFKAP_STR_INITIALIZER_EMPTY; + int32_t AcquisitionLockTimeoutMs = 0; + rd_kafkap_NodeEndpoints_t NodeEndpoints; + NodeEndpoints.NodeEndpoints = NULL; + NodeEndpoints.NodeEndpointCnt = 0; + + rd_kafka_buf_read_throttle_time(rkbuf); + + rd_kafka_buf_read_i16(rkbuf, &ErrorCode); + rd_kafka_buf_read_str(rkbuf, &ErrorStr); + + if (ErrorCode) { + rd_rkb_log(rkb, LOG_ERR, "SHAREFETCH", + "ShareFetch response error %d: '%.*s'", + ErrorCode, + RD_KAFKAP_STR_PR(&ErrorStr)); + return ErrorCode; + } + + rd_kafka_buf_read_i32(rkbuf, &AcquisitionLockTimeoutMs); + + rd_kafka_buf_read_arraycnt(rkbuf, &TopicArrayCnt, RD_KAFKAP_TOPICS_MAX); + /* TODO KIP-932: Check if required. + Verify that TopicArrayCnt seems to be in line with remaining size */ + // rd_kafka_buf_check_len(rkbuf, + // TopicArrayCnt * (3 /*topic min size*/ + + // 4 /*PartitionArrayCnt*/ + 4 + + // 2 + 8 + 4 /*inner header*/)); + + for (i = 0; i < TopicArrayCnt; i++) { + rd_kafkap_str_t topic = RD_ZERO_INIT; + rd_kafka_Uuid_t topic_id = RD_KAFKA_UUID_ZERO; + int32_t PartitionArrayCnt; + int j; + + rd_kafka_buf_read_uuid(rkbuf, &topic_id); + rkt = rd_kafka_topic_find_by_topic_id(rkb->rkb_rk, + topic_id); + if (rkt) + topic = *rkt->rkt_topic; + + rd_kafka_buf_read_arraycnt(rkbuf, &PartitionArrayCnt, + RD_KAFKAP_PARTITIONS_MAX); + + for (j = 0; j < PartitionArrayCnt; j++) { + if (rd_kafka_share_fetch_reply_handle_partition( + rkb, &topic, rkt, rkbuf, request)) + goto err_parse; + } + + if (rkt) { + rd_kafka_topic_destroy0(rkt); + rkt = NULL; + } + /* Topic Tags */ + rd_kafka_buf_skip_tags(rkbuf); + } + + rd_kafka_buf_read_NodeEndpoints(rkbuf, &NodeEndpoints); + + /* Top level tags */ + rd_kafka_buf_skip_tags(rkbuf); + + + if (rd_kafka_buf_read_remain(rkbuf) != 0) { + rd_kafka_buf_parse_fail(rkbuf, + "Remaining data after message set " + "parse: %" PRIusz " bytes", + rd_kafka_buf_read_remain(rkbuf)); + RD_NOTREACHED(); + } + +// done: + RD_IF_FREE(NodeEndpoints.NodeEndpoints, rd_free); + RD_IF_FREE(rkt, rd_kafka_topic_destroy0); + return RD_KAFKA_RESP_ERR_NO_ERROR; + +err_parse: + if (rkt) + rd_kafka_topic_destroy0(rkt); + rd_rkb_dbg(rkb, MSG, "BADMSG", + "Bad message (Fetch v%d): " + "is broker.version.fallback incorrectly set?", + (int)request->rkbuf_reqhdr.ApiVersion); + return rkbuf->rkbuf_err; +} + + +/** + * TODO KIP-932: Implement. + */ +// static void rd_kafak_broker_session_reset(rd_kafka_broker_t *rkb) { +// } +static void rd_kafka_broker_session_update_epoch(rd_kafka_broker_t *rkb) { + if (rkb->rkb_share_fetch_session.epoch == -1) { + rd_kafka_dbg(rkb->rkb_rk, MSG, "SHAREFETCH", + "Not updating next epoch for -1 as it should be -1 again."); + return; + } + if (rkb->rkb_share_fetch_session.epoch == INT32_MAX) + rkb->rkb_share_fetch_session.epoch = 1; + else + rkb->rkb_share_fetch_session.epoch++; +} + +static void rd_kafka_broker_session_add_partition_to_toppars_in_session(rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp) { + rd_kafka_toppar_t *session_rktp, *adding_rktp; + TAILQ_FOREACH(session_rktp, &rkb->rkb_share_fetch_session.toppars_in_session, rktp_rkb_session_link) { + if(rktp == session_rktp) { + rd_kafka_dbg(rkb->rkb_rk, MSG, "SHAREFETCH", + "%s [%" PRId32 + "]: already in ShareFetch session", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition); + return; + } + } + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH", + "%s [%" PRId32 + "]: adding to ShareFetch session", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition); + adding_rktp = rd_kafka_toppar_keep(rktp); + TAILQ_INSERT_TAIL(&rkb->rkb_share_fetch_session.toppars_in_session, adding_rktp, rktp_rkb_session_link); + rkb->rkb_share_fetch_session.toppars_in_session_cnt++; +} + +static void rd_kafka_broker_session_remove_partition_from_toppars_in_session(rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp) { + rd_kafka_toppar_t *session_rktp, *tmp_rktp; + TAILQ_FOREACH_SAFE(session_rktp, &rkb->rkb_share_fetch_session.toppars_in_session, rktp_rkb_session_link, tmp_rktp) { + if(rktp == session_rktp) { + TAILQ_REMOVE(&rkb->rkb_share_fetch_session.toppars_in_session, session_rktp, rktp_rkb_session_link); + rd_kafka_toppar_destroy(session_rktp); // from session list + rkb->rkb_share_fetch_session.toppars_in_session_cnt--; + rd_kafka_dbg(rkb->rkb_rk, MSG, "SHAREFETCH", + "%s [%" PRId32 + "]: removed from ShareFetch session", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition); + return; + } + } + rd_kafka_dbg(rkb->rkb_rk, MSG, "SHAREFETCH", + "%s [%" PRId32 + "]: not found in ShareFetch session", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition); +} + +static void rd_kafka_broker_session_update_toppars_in_session(rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp, rd_bool_t add) { + if(add) + rd_kafka_broker_session_add_partition_to_toppars_in_session(rkb, rktp); + else + rd_kafka_broker_session_remove_partition_from_toppars_in_session(rkb, rktp); + +} + + +// static void rd_kafka_broker_session_update_added_partitions(rd_kafka_broker_t *rkb) { +// size_t i; +// rd_kafka_toppar_t *rktp, *removed_rktp; +// rd_list_t *toppars_to_add = rkb->rkb_share_fetch_session.toppars_to_add; +// rd_list_t *added_toppars = rkb->rkb_share_fetch_session.adding_toppars; + +// if(added_toppars == NULL || rd_list_cnt(added_toppars) == 0) +// return; + +// RD_LIST_FOREACH(rktp, added_toppars, i) { +// rd_kafka_broker_session_update_toppars_in_session(rkb, rktp, rd_true /* add */); +// if(toppars_to_add) { +// removed_rktp = rd_list_remove(toppars_to_add, rktp); +// if(removed_rktp) +// rd_kafka_toppar_destroy(removed_rktp); // from partitions list +// } +// } +// rd_list_destroy(added_toppars); +// rkb->rkb_share_fetch_session.adding_toppars = NULL; +// } + +// static void rd_kafka_broker_session_update_removed_partitions(rd_kafka_broker_t *rkb) { +// size_t i; +// rd_kafka_toppar_t *rktp, *removed_rktp; +// rd_list_t *toppars_to_forget = rkb->rkb_share_fetch_session.toppars_to_forget; +// rd_list_t *forgotten_toppars = rkb->rkb_share_fetch_session.forgetting_toppars; + +// if(forgotten_toppars == NULL || rd_list_cnt(forgotten_toppars) == 0) +// return; + +// RD_LIST_FOREACH(rktp, forgotten_toppars, i) { +// rd_kafka_broker_session_update_toppars_in_session(rkb, rktp, rd_false /* remove */); +// if(toppars_to_forget) { +// removed_rktp = rd_list_remove(toppars_to_forget, rktp); +// if(removed_rktp) +// rd_kafka_toppar_destroy(removed_rktp); // from partitions list +// } +// } +// rd_list_destroy(forgotten_toppars); +// rkb->rkb_share_fetch_session.forgetting_toppars = NULL; +// } + +static void rd_kafka_broker_session_update_toppars_list( + rd_kafka_broker_t *rkb, + rd_list_t **request_toppars_ptr, + rd_list_t **toppars_to_remove_ptr, + rd_bool_t add) { + size_t i; + rd_kafka_toppar_t *rktp, *removed_rktp; + rd_list_t *request_toppars = *request_toppars_ptr; + rd_list_t *toppars_to_remove = *toppars_to_remove_ptr; + + if (request_toppars == NULL || rd_list_cnt(request_toppars) == 0) + return; + + RD_LIST_FOREACH(rktp, request_toppars, i) { + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH","%s [%" PRId32 "], add: %d", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition, + add); + rd_kafka_broker_session_update_toppars_in_session(rkb, rktp, add); + if (toppars_to_remove) { + removed_rktp = rd_list_remove(toppars_to_remove, rktp); + if (removed_rktp) { + rd_kafka_toppar_destroy(removed_rktp); /* from partitions list */ + if(rd_list_empty(toppars_to_remove)) { + rd_list_destroy(toppars_to_remove); + *toppars_to_remove_ptr = NULL; + } + } + } + } + rd_list_destroy(request_toppars); + *request_toppars_ptr = NULL; +} + +static void rd_kafka_broker_session_update_added_partitions( + rd_kafka_broker_t *rkb) { + rd_kafka_broker_session_update_toppars_list( + rkb, &rkb->rkb_share_fetch_session.adding_toppars, + &rkb->rkb_share_fetch_session.toppars_to_add, rd_true); +} + +static void rd_kafka_broker_session_update_removed_partitions( + rd_kafka_broker_t *rkb) { + rd_kafka_broker_session_update_toppars_list( + rkb, &rkb->rkb_share_fetch_session.forgetting_toppars, + &rkb->rkb_share_fetch_session.toppars_to_forget, rd_false); +} + +static void rd_kafka_broker_session_update_partitions(rd_kafka_broker_t *rkb) { + rd_kafka_broker_session_update_added_partitions(rkb); + rd_kafka_broker_session_update_removed_partitions(rkb); +} + + +/** + * Update ShareFetch session state after a Fetch or ShareFetch response. + * TODO KIP-932: Improve efficiency of this function. + */ +static void rd_kafka_broker_session_update(rd_kafka_broker_t *rkb) { + rd_kafka_broker_session_update_epoch(rkb); + rd_kafka_broker_session_update_partitions(rkb); +} + +/** + * @broker ShareFetchResponse handling. + * + * @locality broker thread (or any thread if err == __DESTROY). + */ +static void rd_kafka_broker_share_fetch_reply(rd_kafka_t *rk, + rd_kafka_broker_t *rkb, + rd_kafka_resp_err_t err, + rd_kafka_buf_t *reply, + rd_kafka_buf_t *request, + void *opaque) { + + rd_kafka_op_t *rko_orig = opaque; + + if (err == RD_KAFKA_RESP_ERR__DESTROY) { + /* TODO KIP-932: Check what is needed out of the below */ + rd_kafka_broker_session_update(rkb); + rd_kafka_op_reply(rko_orig, err); + return; /* Terminating */ + } + + rd_kafka_assert(rkb->rkb_rk, rkb->rkb_fetching > 0); + + /* Parse and handle the messages (unless the request errored) */ + if (!err && reply) + err = rd_kafka_share_fetch_reply_handle(rkb, reply, request); + + if (rko_orig) + rd_kafka_op_reply(rko_orig, err); + + rd_kafka_broker_session_update(rkb); + // if (rkb->rkb_share_fetch_session.adding_toppars) + + /* TODO KIP-932: Check if this is the right place for this or after error handling */ + rkb->rkb_fetching = 0; + + if (unlikely(err)) { + char tmp[128]; + + rd_rkb_dbg(rkb, MSG, "FETCH", "Fetch reply: %s", + rd_kafka_err2str(err)); + switch (err) { + case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: + case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: + case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: + case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID: + /* Request metadata information update */ + rd_snprintf(tmp, sizeof(tmp), "FetchRequest failed: %s", + rd_kafka_err2str(err)); + rd_kafka_metadata_refresh_known_topics( + rkb->rkb_rk, NULL, rd_true /*force*/, tmp); + /* FALLTHRU */ + + case RD_KAFKA_RESP_ERR__TRANSPORT: + case RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT: + case RD_KAFKA_RESP_ERR__MSG_TIMED_OUT: + /* The fetch is already intervalled from + * consumer_serve() so dont retry. */ + break; + + default: + break; + } + + rd_kafka_broker_fetch_backoff(rkb, err); + /* FALLTHRU */ + } +} /** * @broker FetchResponse handling. @@ -959,6 +1445,480 @@ static rd_bool_t can_use_topic_ids(rd_kafka_broker_t *rkb) { return rd_true; } + +void rd_kafka_ShareFetchRequest( + rd_kafka_broker_t *rkb, + const rd_kafkap_str_t *group_id, + const rd_kafkap_str_t *member_id, + int32_t share_session_epoch, + int32_t wait_max_ms, + int32_t min_bytes, + int32_t max_bytes, + int32_t max_records, + int32_t batch_size, + rd_list_t *toppars_to_send, + rd_list_t *toppars_to_forget, + rd_bool_t is_leave_request, + rd_kafka_op_t *rko_orig, + rd_ts_t now) { + rd_kafka_toppar_t *rktp; + rd_kafka_buf_t *rkbuf; + int cnt = 0; + size_t of_TopicArrayCnt = 0; + int TopicArrayCnt = 0; + size_t of_PartitionArrayCnt = 0; + int PartitionArrayCnt = 0; + rd_kafka_topic_t *rkt_last = NULL; + int16_t ApiVersion = 0; + size_t rkbuf_size = 0; + int toppars_to_send_cnt = toppars_to_send ? rd_list_cnt(toppars_to_send) : 0; + int i; + size_t j; + rd_bool_t has_acknowledgements_or_topics_to_add = toppars_to_send && rd_list_cnt(toppars_to_send) > 0 ? rd_true : rd_false; + rd_bool_t has_toppars_to_forget = toppars_to_forget && rd_list_cnt(toppars_to_forget) > 0 ? rd_true : rd_false; + rd_bool_t is_fetching_messages = max_records > 0 ? rd_true : rd_false; + + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH", "toppars_to_send_cnt=%d, has_acknowledgements_or_topics_to_add=%d, has_toppars_to_forget=%d, is_fetching_messages=%d", + toppars_to_send_cnt, has_acknowledgements_or_topics_to_add, has_toppars_to_forget, is_fetching_messages); + /* + * Only sending 1 aknowledgement for each partition. StartOffset + LastOffset + AcknowledgementType (ACCEPT for now). + * TODO KIP-932: Change this to accommodate explicit acknowledgements. + */ + size_t acknowledgement_size = 8 + 8 + 1; + + /* Calculate buffer size */ + if (group_id) + rkbuf_size += RD_KAFKAP_STR_SIZE(group_id); + if (member_id) + rkbuf_size += RD_KAFKAP_STR_SIZE(member_id); + /* ShareSessionEpoch + WaitMaxMs + MinBytes + MaxBytes + MaxRecords + BatchSize + TopicArrayCnt*/ + rkbuf_size += 4 + 4 + 4 + 4 + 4 + 4 + 4; + /* N x (topic id + partition id + acknowledgement) */ + rkbuf_size += (toppars_to_send_cnt * (32 + 4 + acknowledgement_size)); + if( has_toppars_to_forget) { + /* M x (topic id + partition id) */ + rkbuf_size += (rd_list_cnt(toppars_to_forget) * (32 + 4)); + } + + ApiVersion = rd_kafka_broker_ApiVersion_supported(rkb, RD_KAFKAP_ShareFetch, + 1, 1, NULL); + + rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_ShareFetch, 1, + rkbuf_size, + rd_true); + + if (rkb->rkb_features & RD_KAFKA_FEATURE_MSGVER2) + rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, + RD_KAFKA_FEATURE_MSGVER2); + else if (rkb->rkb_features & RD_KAFKA_FEATURE_MSGVER1) + rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, + RD_KAFKA_FEATURE_MSGVER1); + else if (rkb->rkb_features & RD_KAFKA_FEATURE_THROTTLETIME) + rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, + RD_KAFKA_FEATURE_THROTTLETIME); + + /* GroupId */ + rd_kafka_buf_write_kstr(rkbuf, group_id); + + /* MemberId */ + rd_kafka_buf_write_kstr(rkbuf, member_id); + + // printf(" --------------------------------------- rd_kafka_ShareFetchRequest: member_id=%.*s\n", + // RD_KAFKAP_STR_PR(member_id)); + + /* ShareSessionEpoch */ + rd_kafka_buf_write_i32(rkbuf, share_session_epoch); + + /* WaitMaxMs */ + rd_kafka_buf_write_i32(rkbuf, wait_max_ms); + + /* MinBytes */ + rd_kafka_buf_write_i32(rkbuf, min_bytes); + + /* MaxBytes */ + rd_kafka_buf_write_i32(rkbuf, max_bytes); + + /* MaxRecords */ + rd_kafka_buf_write_i32(rkbuf, max_records); + + /* BatchSize */ + rd_kafka_buf_write_i32(rkbuf, batch_size); + + /* Write zero TopicArrayCnt but store pointer for later update */ + of_TopicArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); + + RD_LIST_FOREACH(rktp, toppars_to_send, i) { + + /* TODO KIP-932: This condition will cause partitions of same topics + to be inside single instance of the topic as toppars_to_send is not + sorted. Eg: T1 0, T1 1, T2 0, T1 3, T1 5, T2 1 will translate to + T1 (0,1), T2 (0), T1 (3, 5), T2 (1) instead it should be + T1 (0,1,3,5) T2(0,1) Fix this. */ + if (rkt_last != rktp->rktp_rkt) { + if (rkt_last != NULL) { + /* Update PartitionArrayCnt */ + rd_kafka_buf_finalize_arraycnt( + rkbuf, of_PartitionArrayCnt, + PartitionArrayCnt); + /* Topic tags */ + rd_kafka_buf_write_tags_empty(rkbuf); + } + + rd_kafka_topic_rdlock(rktp->rktp_rkt); + /* Topic ID */ + rd_kafka_buf_write_uuid( + rkbuf, &rktp->rktp_rkt->rkt_topic_id); + rd_kafka_topic_rdunlock(rktp->rktp_rkt); + + TopicArrayCnt++; + rkt_last = rktp->rktp_rkt; + /* Partition count */ + of_PartitionArrayCnt = + rd_kafka_buf_write_arraycnt_pos(rkbuf); + PartitionArrayCnt = 0; + } + + PartitionArrayCnt++; + + /* Partition */ + rd_kafka_buf_write_i32(rkbuf, rktp->rktp_partition); + + // printf(" ------------------------------------------------------------------ AcknowledgementBatches for topic %.*s [%" PRId32 "] : first_offset=%" PRId64 ", last_offset=%" PRId64 "\n", + // RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + // rktp->rktp_partition, + // rktp->rktp_share_acknowledge.first_offset, + // rktp->rktp_share_acknowledge.last_offset); + /* AcknowledgementBatches */ + if (rktp->rktp_share_acknowledge_count > 0) { + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", "rd_kafka_ShareFetchRequest: topic %.*s [%" PRId32 "] : sending %ld acknowledgements", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rktp->rktp_share_acknowledge_count); + /* For now we only support ACCEPT */ + rd_kafka_buf_write_arraycnt(rkbuf, rktp->rktp_share_acknowledge_count); /* ArrayCnt = 1 */ + for(j = 0; j < rktp->rktp_share_acknowledge_count; j++) { + /* FirstOffset */ + rd_kafka_buf_write_i64(rkbuf, rktp->rktp_share_acknowledge[j].first_offset); + /* LastOffset */ + rd_kafka_buf_write_i64(rkbuf, rktp->rktp_share_acknowledge[j].last_offset); + /* AcknowledgementType */ + rd_kafka_buf_write_arraycnt(rkbuf, 1); /* ArrayCnt = 1 */ + rd_kafka_buf_write_i8(rkbuf, 1); /* ACCEPT */ + /* Acknowledgement tags */ + rd_kafka_buf_write_tags_empty(rkbuf); + } + rktp->rktp_share_acknowledge_count = 0; + rd_free(rktp->rktp_share_acknowledge); + rktp->rktp_share_acknowledge = NULL; + } else { + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", "rd_kafka_ShareFetchRequest: topic %.*s [%" PRId32 "] : No acknowledgements to send", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + /* No acknowledgements */ + rd_kafka_buf_write_arraycnt(rkbuf, 0); + } + + /* Partition tags */ + rd_kafka_buf_write_tags_empty(rkbuf); + + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", + "Share Fetch topic %.*s [%" PRId32 "]", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + + cnt++; + } + + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH", + "Share Fetch Request with %d toppars on %d topics", + cnt, TopicArrayCnt); + + if (rkt_last != NULL) { + /* Update last topic's PartitionArrayCnt */ + rd_kafka_buf_finalize_arraycnt(rkbuf, of_PartitionArrayCnt, + PartitionArrayCnt); + /* Topic tags */ + rd_kafka_buf_write_tags_empty(rkbuf); + } + + /* Update TopicArrayCnt */ + rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, TopicArrayCnt); + + if(toppars_to_send) { + rd_list_destroy(toppars_to_send); + } + + if(is_leave_request || has_acknowledgements_or_topics_to_add || has_toppars_to_forget || is_fetching_messages) { + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH", + "Share Fetch Request sent with%s%s%s", + has_acknowledgements_or_topics_to_add ? " acknowledgements," : "", + has_toppars_to_forget ? " forgotten toppars," : "", + is_fetching_messages ? " fetching messages" : ""); + } else { + rd_kafka_buf_destroy(rkbuf); + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH", + "Share Fetch Request not sent since there are no " + "acknowledgements, forgotten toppars or messages to fetch"); + rd_kafka_op_reply(rko_orig, RD_KAFKA_RESP_ERR__NOOP); + return; + } + + if (has_toppars_to_forget) { + TopicArrayCnt = 0; + PartitionArrayCnt = 0; + rkt_last = NULL; + /* Write zero TopicArrayCnt but store pointer for later update */ + of_TopicArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH", + "Forgetting %d toppars", rd_list_cnt(toppars_to_forget)); + RD_LIST_FOREACH(rktp, toppars_to_forget, i) { + /* TODO KIP-932: This condition will cause partitions of same topics + to be inside single instance of the topic as toppars_to_send is not + sorted. Eg: T1 0, T1 1, T2 0, T1 3, T1 5, T2 1 will translate to + T1 (0,1), T2 (0), T1 (3, 5), T2 (1) instead it should be + T1 (0,1,3,5) T2(0,1) Fix this. */ + if (rkt_last != rktp->rktp_rkt) { + if (rkt_last != NULL) { + /* Update PartitionArrayCnt */ + rd_kafka_buf_finalize_arraycnt( + rkbuf, of_PartitionArrayCnt, + PartitionArrayCnt); + /* Topic tags */ + rd_kafka_buf_write_tags_empty(rkbuf); + } + + rd_kafka_topic_rdlock(rktp->rktp_rkt); + /* Topic ID */ + rd_kafka_buf_write_uuid( + rkbuf, &rktp->rktp_rkt->rkt_topic_id); + rd_kafka_topic_rdunlock(rktp->rktp_rkt); + + TopicArrayCnt++; + rkt_last = rktp->rktp_rkt; + /* Partition count */ + of_PartitionArrayCnt = + rd_kafka_buf_write_arraycnt_pos(rkbuf); + PartitionArrayCnt = 0; + } + + PartitionArrayCnt++; + + /* Partition */ + rd_kafka_buf_write_i32(rkbuf, rktp->rktp_partition); + + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", + "Forgetting Fetch partition %.*s [%" PRId32 "]", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + + } + if (rkt_last != NULL) { + /* Update last topic's PartitionArrayCnt */ + rd_kafka_buf_finalize_arraycnt(rkbuf, of_PartitionArrayCnt, + PartitionArrayCnt); + /* Topic tags */ + rd_kafka_buf_write_tags_empty(rkbuf); + } + /* Update TopicArrayCnt */ + rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, TopicArrayCnt); + } else { + /* ForgottenToppars */ + rd_kafka_buf_write_arraycnt(rkbuf, 0); + } + + /* Consider Fetch requests blocking if fetch.wait.max.ms >= 1s */ + if (rkb->rkb_rk->rk_conf.fetch_wait_max_ms >= 1000) + rkbuf->rkbuf_flags |= RD_KAFKA_OP_F_BLOCKING; + + /* Use configured timeout */ + rd_kafka_buf_set_timeout(rkbuf, + rkb->rkb_rk->rk_conf.socket_timeout_ms + + rkb->rkb_rk->rk_conf.fetch_wait_max_ms, + now); + + rkb->rkb_fetching = 1; + rd_kafka_dbg(rkb->rkb_rk, MSG, "FETCH", + "Issuing ShareFetch request (max wait %dms, min %d bytes, " + "max %d bytes, max %d records) with %d toppars to broker %s " + "(id %" PRId32 ")", + wait_max_ms, min_bytes, max_bytes, max_records, cnt, + rkb->rkb_name, rkb->rkb_nodeid); + rd_kafka_broker_buf_enq1(rkb, rkbuf, rd_kafka_broker_share_fetch_reply, rko_orig); + + return; +} + +static rd_list_t *rd_kafka_broker_share_fetch_get_toppars_to_send_on_leave(rd_kafka_broker_t *rkb) { + /* TODO KIP-932: Implement this properly. Remaining acknowledgements should be sent */ + + // TAILQ_FOREACH(rktp, &rkb->rkb_share_fetch_session.toppars_in_session, rktp_rkblink) { + // if (rktp->rktp_share_acknowledge.first_offset >= 0) { + // rd_list_add(toppars_to_send, rktp); + // } + // } + + return rd_list_new(0, NULL); +} + +static rd_list_t *rd_kafka_broker_share_fetch_get_toppars_to_send(rd_kafka_broker_t *rkb) { + /* TODO KIP-932: Improve this allocation with Acknowledgement implementation */ + int adding_toppar_cnt = rkb->rkb_share_fetch_session.toppars_to_add ? rd_list_cnt(rkb->rkb_share_fetch_session.toppars_to_add) : 0; + int intial_toppars_to_send_cnt = rkb->rkb_toppar_cnt + adding_toppar_cnt; + rd_list_t *toppars_to_send = rd_list_new(intial_toppars_to_send_cnt, NULL); + rd_kafka_toppar_t *rktp; + int i; + + TAILQ_FOREACH(rktp, &rkb->rkb_share_fetch_session.toppars_in_session, rktp_rkb_session_link) { + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", "rd_kafka_broker_share_fetch_get_toppars_to_send: checking toppar topic %.*s [%" PRId32 "] with %ld acknowledgements", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rktp->rktp_share_acknowledge_count); + if (rktp->rktp_share_acknowledge_count > 0) { + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", "rd_kafka_broker_share_fetch_get_toppars_to_send: adding to toppars_to_send topic %.*s [%" PRId32 "] with %ld acknowledgements", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition, + rktp->rktp_share_acknowledge_count); + rd_list_add(toppars_to_send, rktp); + } else { + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", "rd_kafka_broker_share_fetch_get_toppars_to_send: not adding to toppars_to_send topic %.*s [%" PRId32 "] since it has no acknowledgements", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + } + } + + if(rkb->rkb_share_fetch_session.toppars_to_add) { + RD_LIST_FOREACH(rktp, rkb->rkb_share_fetch_session.toppars_to_add, i) { + rd_rkb_dbg(rkb, FETCH, "SHAREFETCH", "rd_kafka_broker_share_fetch_get_toppars_to_send: adding topic %.*s [%" PRId32 "] to the session", + RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), + rktp->rktp_partition); + rd_list_add(toppars_to_send, rktp); + } + } + + return toppars_to_send; +} + +void rd_kafka_broker_share_fetch_session_clear(rd_kafka_broker_t *rkb) { + rd_kafka_toppar_t *rktp, *tmp_rktp; + + rkb->rkb_share_fetch_session.epoch = -1; + + /* Clear toppars in session */ + TAILQ_FOREACH_SAFE(rktp, &rkb->rkb_share_fetch_session.toppars_in_session, rktp_rkb_session_link, tmp_rktp) { + TAILQ_REMOVE(&rkb->rkb_share_fetch_session.toppars_in_session, rktp, rktp_rkb_session_link); + if(rktp->rktp_share_acknowledge) { + rd_free(rktp->rktp_share_acknowledge); + rktp->rktp_share_acknowledge = NULL; + rktp->rktp_share_acknowledge_count = 0; + } + rd_kafka_toppar_destroy(rktp); // from session list + rd_rkb_dbg(rkb, BROKER, "SHAREFETCH", + "%s [%" PRId32 + "]: removed from ShareFetch session on clear", + rktp->rktp_rkt->rkt_topic->str, + rktp->rktp_partition); + } + rkb->rkb_share_fetch_session.toppars_in_session_cnt = 0; + + /* Clear toppars to add */ + if(rkb->rkb_share_fetch_session.toppars_to_add) { + rd_rkb_dbg(rkb, BROKER, "SHAREFETCH", + "Clearing %d toppars to add from ShareFetch session on clear", + rd_list_cnt(rkb->rkb_share_fetch_session.toppars_to_add)); + rd_list_destroy(rkb->rkb_share_fetch_session.toppars_to_add); + rkb->rkb_share_fetch_session.toppars_to_add = NULL; + } + + /* Clear toppars to forget */ + if(rkb->rkb_share_fetch_session.toppars_to_forget) { + rd_rkb_dbg(rkb, BROKER, "SHAREFETCH", + "Clearing %d toppars to forget from ShareFetch session on clear", + rd_list_cnt(rkb->rkb_share_fetch_session.toppars_to_forget)); + rd_list_destroy(rkb->rkb_share_fetch_session.toppars_to_forget); + rkb->rkb_share_fetch_session.toppars_to_forget = NULL; + } + + /* Clear adding toppars */ + if(rkb->rkb_share_fetch_session.adding_toppars) { + rd_rkb_dbg(rkb, BROKER, "SHAREFETCH", + "Clearing %d adding toppars from ShareFetch session on clear", + rd_list_cnt(rkb->rkb_share_fetch_session.adding_toppars)); + rd_list_destroy(rkb->rkb_share_fetch_session.adding_toppars); + rkb->rkb_share_fetch_session.adding_toppars = NULL; + } + + /* Clear forgetting toppars */ + if(rkb->rkb_share_fetch_session.forgetting_toppars) { + rd_rkb_dbg(rkb, BROKER, "SHAREFETCH", + "Clearing %d forgetting toppars from ShareFetch session on clear", + rd_list_cnt(rkb->rkb_share_fetch_session.forgetting_toppars)); + rd_list_destroy(rkb->rkb_share_fetch_session.forgetting_toppars); + rkb->rkb_share_fetch_session.forgetting_toppars = NULL; + } +} + +void rd_kafka_broker_share_fetch_leave(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko_orig, rd_ts_t now) { + rd_kafka_cgrp_t *rkcg = rkb->rkb_rk->rk_cgrp; + rd_assert(rkb->rkb_rk->rk_cgrp); + rd_kafka_ShareFetchRequest( + rkb, + rkcg->rkcg_group_id, /* group_id */ + rkcg->rkcg_member_id, /* member_id */ + rkb->rkb_share_fetch_session.epoch, /* share_session_epoch */ + rkb->rkb_rk->rk_conf.fetch_wait_max_ms, + rkb->rkb_rk->rk_conf.fetch_min_bytes, + rkb->rkb_rk->rk_conf.fetch_max_bytes, + 0, + 0, + rd_kafka_broker_share_fetch_get_toppars_to_send_on_leave(rkb), /* toppars to send */ + NULL, /* forgetting toppars */ + rd_true, /* leave request */ + rko_orig, /* rko */ + now); + rd_kafka_broker_share_fetch_session_clear(rkb); +} + +void rd_kafka_broker_share_fetch(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko_orig, rd_ts_t now) { + + rd_kafka_cgrp_t *rkcg = rkb->rkb_rk->rk_cgrp; + int32_t max_records = 0; + + /* TODO KIP-932: Check if needed while closing the consumer.*/ + rd_assert(rkb->rkb_rk->rk_cgrp); + + if(!rkcg->rkcg_member_id) { + rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH", + "Share Fetch requested without member_id"); + rd_kafka_op_reply(rko_orig, RD_KAFKA_RESP_ERR__INVALID_ARG); + return; + } + + if(rko_orig->rko_u.share_fetch.should_fetch) { + max_records = 500; + } + + if(rkb->rkb_share_fetch_session.toppars_to_add) + rkb->rkb_share_fetch_session.adding_toppars = rd_list_copy(rkb->rkb_share_fetch_session.toppars_to_add, rd_kafka_toppar_list_copy, NULL); + if(rkb->rkb_share_fetch_session.toppars_to_forget) + rkb->rkb_share_fetch_session.forgetting_toppars = rd_list_copy(rkb->rkb_share_fetch_session.toppars_to_forget, rd_kafka_toppar_list_copy, NULL); + + rd_kafka_ShareFetchRequest( + rkb, + rkcg->rkcg_group_id, /* group_id */ + rkcg->rkcg_member_id, /* member_id */ + rkb->rkb_share_fetch_session.epoch, /* share_session_epoch */ + rkb->rkb_rk->rk_conf.fetch_wait_max_ms, + rkb->rkb_rk->rk_conf.fetch_min_bytes, + rkb->rkb_rk->rk_conf.fetch_max_bytes, + max_records, + 500, + rd_kafka_broker_share_fetch_get_toppars_to_send(rkb), /* toppars to send */ + rkb->rkb_share_fetch_session.toppars_to_forget, /* forgetting toppars */ + rd_false, /* not leave request */ + rko_orig, /* rko */ + now); +} + /** * @brief Build and send a Fetch request message for all underflowed toppars * for a specific broker. @@ -1078,6 +2038,11 @@ int rd_kafka_broker_fetch_toppars(rd_kafka_broker_t *rkb, rd_ts_t now) { /* Topic tags */ rd_kafka_buf_write_tags_empty(rkbuf); } + + /* TODO: This is not thread safe as topic can + be recreated in which case topic id is + updated from the main thread and we are + sending topic id from broker thread.*/ if (rd_kafka_buf_ApiVersion(rkbuf) > 12) { /* Topic id must be non-zero here */ rd_dassert(!RD_KAFKA_UUID_IS_ZERO( diff --git a/src/rdkafka_fetcher.h b/src/rdkafka_fetcher.h index e304f1369f..ee230d204a 100644 --- a/src/rdkafka_fetcher.h +++ b/src/rdkafka_fetcher.h @@ -40,5 +40,8 @@ rd_ts_t rd_kafka_toppar_fetch_decide(rd_kafka_toppar_t *rktp, rd_kafka_broker_t *rkb, int force_remove); +void rd_kafka_broker_share_fetch_leave(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko_orig, rd_ts_t now); +void rd_kafka_broker_share_fetch(rd_kafka_broker_t *rkb, rd_kafka_op_t *rko_orig, rd_ts_t now); + #endif /* _RDKAFKA_FETCHER_H_ */ diff --git a/src/rdkafka_int.h b/src/rdkafka_int.h index d8370ff599..c9fe9da3e1 100644 --- a/src/rdkafka_int.h +++ b/src/rdkafka_int.h @@ -94,6 +94,9 @@ typedef struct rd_kafka_lwtopic_s rd_kafka_lwtopic_t; #define RD_KAFKA_OFFSET_IS_LOGICAL(OFF) ((OFF) < 0) +#define RD_KAFKA_IS_SHARE_CONSUMER(rk) \ + ((rk)->rk_type == RD_KAFKA_CONSUMER && \ + (rk)->rk_conf.share.is_share_consumer) /** * @struct Represents a fetch position: @@ -1257,4 +1260,8 @@ int rd_kafka_rebootstrap_tmr_stop(rd_kafka_t *rk); void rd_kafka_reset_any_broker_down_reported(rd_kafka_t *rk); +rd_kafka_op_res_t rd_kafka_share_fetch_fanout_op(rd_kafka_t *rk, + rd_kafka_q_t *rkq, + rd_kafka_op_t *rko); + #endif /* _RDKAFKA_INT_H_ */ diff --git a/src/rdkafka_msgset_reader.c b/src/rdkafka_msgset_reader.c index 451dd35442..cc3c590d00 100644 --- a/src/rdkafka_msgset_reader.c +++ b/src/rdkafka_msgset_reader.c @@ -1422,6 +1422,8 @@ rd_kafka_msgset_reader_run(rd_kafka_msgset_reader_t *msetr) { err = RD_KAFKA_RESP_ERR_NO_ERROR; } + // printf(" +++++++++++++++++++ Received %d messages\n", msetr->msetr_msgcnt); + rd_rkb_dbg(msetr->msetr_rkb, MSG | RD_KAFKA_DBG_FETCH, "CONSUME", "Enqueue %i %smessage(s) (%" PRId64 " bytes, %d ops) on %s [%" PRId32 diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 5dbbf9c9d4..0cce441768 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -122,7 +122,13 @@ const char *rd_kafka_op2str(rd_kafka_op_type_t type) { "REPLY:RD_KAFKA_OP_SET_TELEMETRY_BROKER", [RD_KAFKA_OP_TERMINATE_TELEMETRY] = "REPLY:RD_KAFKA_OP_TERMINATE_TELEMETRY", - [RD_KAFKA_OP_ELECTLEADERS] = "REPLY:ELECTLEADERS", + [RD_KAFKA_OP_ELECTLEADERS] = "REPLY:ELECTLEADERS", + [RD_KAFKA_OP_SHARE_FETCH] = "REPLY:SHARE_FETCH", + [RD_KAFKA_OP_SHARE_FETCH_FANOUT] = "REPLY:SHARE_FETCH_FANOUT", + [RD_KAFKA_OP_SHARE_SESSION_PARTITION_ADD] = + "REPLY:SHARE_SESSION_PARTITION_ADD", + [RD_KAFKA_OP_SHARE_SESSION_PARTITION_REMOVE] = + "REPLY:SHARE_SESSION_PARTITION_REMOVE", }; if (type & RD_KAFKA_OP_REPLY) @@ -287,6 +293,13 @@ rd_kafka_op_t *rd_kafka_op_new0(const char *source, rd_kafka_op_type_t type) { sizeof(rko->rko_u.telemetry_broker), [RD_KAFKA_OP_TERMINATE_TELEMETRY] = _RD_KAFKA_OP_EMPTY, [RD_KAFKA_OP_ELECTLEADERS] = sizeof(rko->rko_u.admin_request), + [RD_KAFKA_OP_SHARE_FETCH] = sizeof(rko->rko_u.share_fetch), + [RD_KAFKA_OP_SHARE_FETCH_FANOUT] = + sizeof(rko->rko_u.share_fetch_fanout), + [RD_KAFKA_OP_SHARE_SESSION_PARTITION_ADD] = + _RD_KAFKA_OP_EMPTY, + [RD_KAFKA_OP_SHARE_SESSION_PARTITION_REMOVE] = + _RD_KAFKA_OP_EMPTY, }; size_t tsize = op2size[type & ~RD_KAFKA_OP_FLAGMASK]; @@ -507,6 +520,15 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { rd_kafka_broker_destroy); break; + case RD_KAFKA_OP_SHARE_FETCH: + RD_IF_FREE(rko->rko_u.share_fetch.target_broker, + rd_kafka_broker_destroy); + break; + + case RD_KAFKA_OP_SHARE_FETCH_FANOUT: + /* No heap-allocated resources to clean up */ + break; + default: break; } diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index e79309aa02..ef0e4c4d73 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -189,6 +189,13 @@ typedef enum { RD_KAFKA_OP_ELECTLEADERS, /**< Admin: * ElectLeaders * u.admin_request */ + RD_KAFKA_OP_SHARE_FETCH, /**< broker op: Issue share fetch request if + applicable. */ + RD_KAFKA_OP_SHARE_FETCH_FANOUT, /**< fanout share fetch operation */ + RD_KAFKA_OP_SHARE_SESSION_PARTITION_ADD, /**< share session: + * add partition */ + RD_KAFKA_OP_SHARE_SESSION_PARTITION_REMOVE, /**< share session: + * remove partition */ RD_KAFKA_OP__END } rd_kafka_op_type_t; @@ -724,6 +731,31 @@ struct rd_kafka_op_s { void (*cb)(rd_kafka_t *rk, void *rkb); } terminated; + struct { + + rd_bool_t should_leave; /**< Whether this broker should + * leave the share-fetch + * session. */ + + /** Whether this broker should share-fetch nonzero + * messages. */ + rd_bool_t should_fetch; + + /** Absolute timeout left to complete this share-fetch. + * TODO KIP-932: Use timeout properly. + */ + rd_ts_t abs_timeout; + + /** Target broker to which op is sent. */ + rd_kafka_broker_t *target_broker; + } share_fetch; + + struct { + /** Absolute timeout for share fetch fanout operation. + */ + rd_ts_t abs_timeout; + } share_fetch_fanout; + } rko_u; }; diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c index ce4f01b467..8ba5a4f7f9 100644 --- a/src/rdkafka_partition.c +++ b/src/rdkafka_partition.c @@ -294,6 +294,9 @@ rd_kafka_toppar_t *rd_kafka_toppar_new0(rd_kafka_topic_t *rkt, rkt->rkt_topic->str, rktp->rktp_partition, rktp, &rktp->rktp_refcnt, func, line); + rktp->rktp_share_acknowledge = NULL; + rktp->rktp_share_acknowledge_count = 0; + return rd_kafka_toppar_keep(rktp); } @@ -336,6 +339,8 @@ void rd_kafka_toppar_destroy_final(rd_kafka_toppar_t *rktp) { /* Clear queues */ rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rd_kafka_msgq_len(&rktp->rktp_xmit_msgq) == 0); + rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_share_acknowledge == NULL); + rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_share_acknowledge_count == 0); rd_kafka_dr_msgq(rktp->rktp_rkt, &rktp->rktp_msgq, RD_KAFKA_RESP_ERR__DESTROY); rd_kafka_q_destroy_owner(rktp->rktp_fetchq); @@ -2594,6 +2599,46 @@ void rd_kafka_toppar_leader_unavailable(rd_kafka_toppar_t *rktp, rd_false /* don't force */); } +/** + * @locality any + */ +rd_bool_t rd_kafka_toppar_is_on_cgrp(rd_kafka_toppar_t *rktp, rd_bool_t do_lock) { + rd_bool_t on_cgrp; + if (do_lock) { + rd_kafka_toppar_lock(rktp); + } + on_cgrp = (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_CGRP) ? rd_true + : rd_false; + + if (do_lock) { + rd_kafka_toppar_unlock(rktp); + } + + return on_cgrp; +} + +/** + * @locality broker thread + */ +static rd_bool_t rd_kafka_toppar_share_are_acknowledgements_present(rd_kafka_toppar_t *rktp) { + return rktp->rktp_share_acknowledge_count > 0 ? rd_true : rd_false; +} + +rd_bool_t rd_kafka_toppar_share_is_valid_to_send_for_fetch(rd_kafka_toppar_t *rktp) { + if (rd_kafka_toppar_share_are_acknowledgements_present(rktp)) { + return rd_true; + } + return rd_kafka_toppar_is_on_cgrp(rktp, rd_true /*do_lock*/); +} + + +/** + * @brief Toppar copier for rd_list_copy() + */ +void *rd_kafka_toppar_list_copy(const void *elem, void *opaque) { + return rd_kafka_toppar_keep((rd_kafka_toppar_t *)elem); +} + const char * rd_kafka_topic_partition_topic(const rd_kafka_topic_partition_t *rktpar) { diff --git a/src/rdkafka_partition.h b/src/rdkafka_partition.h index 97a704f03b..e090194126 100644 --- a/src/rdkafka_partition.h +++ b/src/rdkafka_partition.h @@ -145,6 +145,8 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ rktp_txnlink; /**< rd_kafka_t.rk_eos. * txn_pend_rktps * or txn_rktps */ + TAILQ_ENTRY(rd_kafka_toppar_s) rktp_rkb_session_link; /* rkb_share_fetch_session + * toppars_in_session link */ rd_kafka_topic_t *rktp_rkt; /**< This toppar's topic object */ int32_t rktp_partition; // LOCK: toppar_lock() + topic_wrlock() @@ -478,6 +480,19 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */ rd_atomic64_t rx_ver_drops; /**< Consumer: outdated message * drops. */ } rktp_c; + + /* + * TODO KIP-932: Change this according to need. Currently very basic. Not even handling GAP. + * Sends ACCEPT blindly with implicit acknowledgement. + */ + + /* Dynamic array of acknowledge entries: NULL until allocated. */ + struct rd_kafka_toppar_share_ack_entry { + int64_t first_offset; + int64_t last_offset; + int16_t delivery_count; + } *rktp_share_acknowledge; /* NULL = not initialized */ + size_t rktp_share_acknowledge_count; /* number of entries in rktp_share_acknowledge (0 when NULL) */ }; /** @@ -533,10 +548,20 @@ void rd_kafka_toppar_destroy_final(rd_kafka_toppar_t *rktp); #define rd_kafka_toppar_destroy(RKTP) \ do { \ rd_kafka_toppar_t *_RKTP = (RKTP); \ - if (unlikely(rd_refcnt_sub(&_RKTP->rktp_refcnt) == 0)) \ - rd_kafka_toppar_destroy_final(_RKTP); \ + rd_kafka_toppar_destroy0(__FUNCTION__, __LINE__, _RKTP); \ } while (0) +/* Common destroy helper used by both the macro and the free-wrapper. */ +static RD_UNUSED RD_INLINE void rd_kafka_toppar_destroy0(const char *func, int line, rd_kafka_toppar_t *rktp) { + if (unlikely(rd_refcnt_sub_fl(func, line, &rktp->rktp_refcnt) == 0)) + rd_kafka_toppar_destroy_final(rktp); +} + +/* Free-function compatible wrapper for rd_list_new and similar APIs + * (signature: void (*)(void *)). */ +static RD_UNUSED RD_INLINE void rd_kafka_toppar_destroy_free(void *ptr) { + rd_kafka_toppar_destroy0(__FUNCTION__, __LINE__, (rd_kafka_toppar_t *)ptr); +} #define rd_kafka_toppar_lock(rktp) mtx_lock(&(rktp)->rktp_lock) @@ -677,6 +702,10 @@ rd_kafka_toppars_pause_resume(rd_kafka_t *rk, int flag, rd_kafka_topic_partition_list_t *partitions); +rd_bool_t rd_kafka_toppar_is_on_cgrp(rd_kafka_toppar_t *rktp, rd_bool_t do_lock); +rd_bool_t rd_kafka_toppar_share_is_valid_to_send_for_fetch(rd_kafka_toppar_t *rktp); +void *rd_kafka_toppar_list_copy(const void *elem, void *opaque); + rd_kafka_topic_partition_t *rd_kafka_topic_partition_new(const char *topic, int32_t partition); diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h index 02565ecb3b..8d8362675c 100644 --- a/src/rdkafka_proto.h +++ b/src/rdkafka_proto.h @@ -175,6 +175,8 @@ static RD_UNUSED const char *rd_kafka_ApiKey2str(int16_t ApiKey) { [RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribe", [RD_KAFKAP_GetTelemetrySubscriptions] = "GetTelemetrySubscriptions", [RD_KAFKAP_PushTelemetry] = "PushTelemetry", + [RD_KAFKAP_ShareGroupHeartbeat] = "ShareGroupHeartbeat", + [RD_KAFKAP_ShareFetch] = "ShareFetch", }; static RD_TLS char ret[64]; diff --git a/src/rdkafka_protocol.h b/src/rdkafka_protocol.h index 19190e1447..6064743561 100644 --- a/src/rdkafka_protocol.h +++ b/src/rdkafka_protocol.h @@ -120,8 +120,11 @@ #define RD_KAFKAP_GetTelemetrySubscriptions 71 #define RD_KAFKAP_PushTelemetry 72 #define RD_KAFKAP_AssignReplicasToDirs 73 - -#define RD_KAFKAP__NUM 74 +#define RD_KAFKAP_ShareGroupHeartbeat 76 +#define RD_KAFKAP_ShareGroupDescribe 77 +#define RD_KAFKAP_ShareFetch 78 +#define RD_KAFKAP_ShareAcknowledge 79 +#define RD_KAFKAP__NUM 80 #define RD_KAFKAP_RPC_VERSION_MAX INT16_MAX diff --git a/src/rdkafka_queue.c b/src/rdkafka_queue.c index 92eddccae9..1d25d984d3 100644 --- a/src/rdkafka_queue.c +++ b/src/rdkafka_queue.c @@ -869,6 +869,122 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq, return cnt; } +int rd_kafka_q_serve_share_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); + struct rd_kafka_op_tailq ctrl_msg_q = + TAILQ_HEAD_INITIALIZER(ctrl_msg_q); + rd_kafka_op_t *rko, *next; + rd_kafka_t *rk = rkq->rkq_rk; + rd_kafka_q_t *fwdq; + rd_ts_t abs_timeout; + + mtx_lock(&rkq->rkq_lock); + if ((fwdq = rd_kafka_q_fwd_get(rkq, 0))) { + /* 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_share_rkmessages(fwdq, timeout_ms, rkmessages, + rkmessages_size); + rd_kafka_q_destroy(fwdq); + return cnt; + } + + mtx_unlock(&rkq->rkq_lock); + + abs_timeout = rd_timeout_init(timeout_ms); + + rd_kafka_app_poll_start(rk, rkq, 0, timeout_ms); + + rd_kafka_yield_thread = 0; + while (cnt < rkmessages_size) { + rd_kafka_op_res_t res; + + mtx_lock(&rkq->rkq_lock); + + while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) && + !rd_kafka_q_check_yield(rkq) && + /* Only do a timed wait if no messages are ready, if we + have gotten even one message, just return with it. */ + cnt == 0 && + cnd_timedwait_abs(&rkq->rkq_cond, &rkq->rkq_lock, + abs_timeout) == thrd_success) + ; + + rd_kafka_q_mark_served(rkq); + + 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 non-FETCH callbacks */ + res = + rd_kafka_poll_cb(rk, rkq, rko, RD_KAFKA_Q_CB_RETURN, NULL); + if (res == RD_KAFKA_OP_RES_KEEP || + res == RD_KAFKA_OP_RES_HANDLED) { + /* Callback served, rko is destroyed (if HANDLED). */ + continue; + } else if (unlikely(res == RD_KAFKA_OP_RES_YIELD || + rd_kafka_yield_thread)) { + /* Yield. */ + break; + } + rd_dassert(res == RD_KAFKA_OP_RES_PASS); + + /* If this is a control messages, don't return message to + * application. Add it to a tmp queue from where we can store + * the offset and destroy the op */ + if (unlikely(rd_kafka_op_is_ctrl_msg(rko))) { + TAILQ_INSERT_TAIL(&ctrl_msg_q, rko, rko_link); + continue; + } + + /* Get rkmessage from rko and append to array. */ + rkmessages[cnt++] = rd_kafka_message_get(rko); + } + + /* NOTE: KIP-932: + * For a share consumer, we are not using version barriers, and ideally, + * tmpq should be empty. However, the discard code is retained as + * non-share-consumer might still be around. This assert exists to spot + * any issues as they arise during testing.*/ + rd_dassert(TAILQ_EMPTY(&tmpq)); + + /* 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); + } + + /* Discard ctrl msgs */ + next = TAILQ_FIRST(&ctrl_msg_q); + while (next) { + rko = next; + next = TAILQ_NEXT(next, rko_link); + rd_kafka_op_destroy(rko); + } + + rd_kafka_app_polled(rk, rkq); + + return cnt; +} + void rd_kafka_queue_destroy(rd_kafka_queue_t *rkqu) { diff --git a/src/rdkafka_queue.h b/src/rdkafka_queue.h index ff1a465770..cfed32413a 100644 --- a/src/rdkafka_queue.h +++ b/src/rdkafka_queue.h @@ -878,6 +878,10 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size); +int rd_kafka_q_serve_share_rkmessages(rd_kafka_q_t *rkq, + int timeout_ms, + rd_kafka_message_t **rkmessages, + size_t rkmessages_size); rd_kafka_resp_err_t rd_kafka_q_wait_result(rd_kafka_q_t *rkq, int timeout_ms); int rd_kafka_q_apply(rd_kafka_q_t *rkq, diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 663a07eae3..544ee4b5ac 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -562,6 +562,8 @@ int rd_kafka_buf_read_NodeEndpoints(rd_kafka_buf_t *rkbuf, int32_t i; rd_kafka_buf_read_arraycnt(rkbuf, &NodeEndpoints->NodeEndpointCnt, RD_KAFKAP_BROKERS_MAX); + // printf(" --------------------------------------- rd_kafka_buf_read_NodeEndpoints: NodeEndpointCnt=%d\n", + // NodeEndpoints->NodeEndpointCnt); rd_dassert(!NodeEndpoints->NodeEndpoints); NodeEndpoints->NodeEndpoints = rd_calloc(NodeEndpoints->NodeEndpointCnt, @@ -2442,7 +2444,101 @@ void rd_kafka_ConsumerGroupHeartbeatRequest( rd_kafkap_str_destroy(subscribed_topic_regex_to_send); } +void rd_kafka_ShareGroupHeartbeatRequest( + rd_kafka_broker_t *rkb, + const rd_kafkap_str_t *group_id, + const rd_kafkap_str_t *member_id, + int32_t member_epoch, + const rd_kafkap_str_t *rack_id, + const rd_kafka_topic_partition_list_t *subscribed_topics, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque) { + rd_kafka_buf_t *rkbuf; + int16_t ApiVersion = 0; + int features; + size_t rkbuf_size = 0; + + ApiVersion = rd_kafka_broker_ApiVersion_supported( + rkb, RD_KAFKAP_ShareGroupHeartbeat, 1, 1, &features); + + rd_rkb_dbg(rkb, CGRP, "SHAREHEARTBEAT", + "ShareGroupHeartbeat version %d for group \"%s\", member id " + "\"%s\", topic count = %d", + ApiVersion, group_id ? group_id->str : "NULL", + member_id ? member_id->str : "NULL", + subscribed_topics ? subscribed_topics->cnt : -1); + + if (ApiVersion == -1) { + rd_kafka_cgrp_coord_dead(rkb->rkb_rk->rk_cgrp, + RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE, + "ShareGroupHeartbeatRequest not " + "supported by broker"); + return; + } + + // debug log all the fields + if (rd_rkb_is_dbg(rkb, CGRP)) { + char subscribed_topics_str[512] = "NULL"; + if (subscribed_topics) { + rd_kafka_topic_partition_list_str( + subscribed_topics, subscribed_topics_str, + sizeof(subscribed_topics_str), 0); + } + rd_rkb_dbg(rkb, CGRP, "SHAREHEARTBEAT", + "ShareGroupHeartbeat of group id \"%s\", " + "member id \"%s\", member epoch %d, rack id \"%s\"" + ", subscribed topics \"%s\"", + group_id ? group_id->str : "NULL", + member_id ? member_id->str : "NULL", member_epoch, + rack_id ? rack_id->str : "NULL", + subscribed_topics_str); + } + + if (group_id) + rkbuf_size += RD_KAFKAP_STR_SIZE(group_id); + if (member_id) + rkbuf_size += RD_KAFKAP_STR_SIZE(member_id); + rkbuf_size += 4; /* MemberEpoch */ + if (rack_id) + rkbuf_size += RD_KAFKAP_STR_SIZE(rack_id); + if (subscribed_topics) { + rkbuf_size += + ((subscribed_topics->cnt * (4 + 50)) + 4 /* array size */); + } + + rkbuf = rd_kafka_buf_new_flexver_request( + rkb, RD_KAFKAP_ShareGroupHeartbeat, 1, rkbuf_size, rd_true); + + rd_kafka_buf_write_kstr(rkbuf, group_id); + rd_kafka_buf_write_kstr(rkbuf, member_id); + rd_kafka_buf_write_i32(rkbuf, member_epoch); + rd_kafka_buf_write_kstr(rkbuf, rack_id); + if (subscribed_topics) { + int topics_cnt = subscribed_topics->cnt; + + /* write Topics */ + rd_kafka_buf_write_arraycnt(rkbuf, topics_cnt); + while (--topics_cnt >= 0) { + if (rd_rkb_is_dbg(rkb, CGRP)) + rd_rkb_dbg( + rkb, CGRP, "SHAREHEARTBEAT", + "ShareGroupHeartbeat subscribed " + "topic %s", + subscribed_topics->elems[topics_cnt].topic); + rd_kafka_buf_write_str( + rkbuf, subscribed_topics->elems[topics_cnt].topic, + -1); + } + } else { + rd_kafka_buf_write_arraycnt(rkbuf, -1); + } + + rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, features); + + rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque); +} /** * @brief Construct and send ListGroupsRequest to \p rkb diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index c508ffdaaf..12b2bcbd5f 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -400,6 +400,17 @@ void rd_kafka_ConsumerGroupHeartbeatRequest( rd_kafka_resp_cb_t *resp_cb, void *opaque); +void rd_kafka_ShareGroupHeartbeatRequest( + rd_kafka_broker_t *rkb, + const rd_kafkap_str_t *group_id, + const rd_kafkap_str_t *member_id, + int32_t member_epoch, + const rd_kafkap_str_t *rack_id, + const rd_kafka_topic_partition_list_t *subscribed_topics, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque); + rd_kafka_resp_err_t rd_kafka_MetadataRequest(rd_kafka_broker_t *rkb, const rd_list_t *topics, rd_list_t *topic_ids, diff --git a/tests/0170-share_consumer_subscription.c b/tests/0170-share_consumer_subscription.c new file mode 100644 index 0000000000..eeb7a6e261 --- /dev/null +++ b/tests/0170-share_consumer_subscription.c @@ -0,0 +1,885 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2025, Confluent Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +#include "test.h" + +/** + * Subscription introspection: + * Subscribe to 3 topics, verify subscription(), then unsubscribe, verify empty. + */ +static void test_subscription_introspection(void) { + char errstr[512]; + const char *group = "share-group-sub-introspect"; + const char *t1 = "0154-share-sub-intro-1"; + const char *t2 = "0154-share-sub-intro-2"; + const char *t3 = "0154-share-sub-intro-3"; + + test_create_topic_wait_exists(NULL, t1, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, t2, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, t3, 1, -1, 30 * 1000); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + rd_kafka_t *c = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(c, "%s", errstr); + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(3); + rd_kafka_topic_partition_list_add(subs, t1, RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs, t2, RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs, t3, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(c, subs), "subscribe failed"); + rd_kafka_topic_partition_list_destroy(subs); + + rd_kafka_topic_partition_list_t *cur = NULL; + TEST_ASSERT(!rd_kafka_subscription(c, &cur) && cur, + "subscription() failed"); + TEST_ASSERT(cur->cnt == 3, "expected 3 topics, got %d", cur->cnt); + rd_kafka_topic_partition_list_destroy(cur); + + TEST_ASSERT(!rd_kafka_unsubscribe(c), "unsubscribe failed"); + + cur = NULL; + TEST_ASSERT(!rd_kafka_subscription(c, &cur) && cur, + "subscription() after unsubscribe failed"); + TEST_ASSERT(cur->cnt == 0, + "expected 0 topics after unsubscribe, got %d", cur->cnt); + rd_kafka_topic_partition_list_destroy(cur); + + rd_kafka_consumer_close(c); + rd_kafka_destroy(c); +} + +/** + * Unsubscribe idempotence: + * First unsubscribe empties subscription, second is no-op. + */ +static void test_unsubscribe_idempotence(void) { + char errstr[512]; + const char *group = "share-group-unsub-idem"; + const char *t1 = "0154-share-unsub-idem-1"; + const char *t2 = "0154-share-unsub-idem-2"; + + test_create_topic_wait_exists(NULL, t1, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, t2, 1, -1, 30 * 1000); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + rd_kafka_t *c = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(c, "%s", errstr); + + const char *grp_conf[] = {"share.auto.offset.reset", "SET", "earliest"}; + test_IncrementalAlterConfigs_simple(c, RD_KAFKA_RESOURCE_GROUP, group, + grp_conf, 1); + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(2); + rd_kafka_topic_partition_list_add(subs, t1, RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs, t2, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(c, subs), "subscribe failed"); + rd_kafka_topic_partition_list_destroy(subs); + + TEST_ASSERT(!rd_kafka_unsubscribe(c), "first unsubscribe failed"); + TEST_ASSERT(!rd_kafka_unsubscribe(c), + "second unsubscribe should be idempotent"); + + rd_kafka_topic_partition_list_t *cur = NULL; + TEST_ASSERT(!rd_kafka_subscription(c, &cur) && cur, + "subscription() failed"); + TEST_ASSERT(cur->cnt == 0, + "expected 0 after double unsubscribe, got %d", cur->cnt); + rd_kafka_topic_partition_list_destroy(cur); + + test_delete_topic(c, t1); + test_delete_topic(c, t2); + rd_kafka_consumer_close(c); + rd_kafka_destroy(c); +} + +/** + * Resubscribe replacing set (A,B) -> (C,D) verifies old topics gone. + */ +static void test_resubscribe_replaces_set(void) { + char errstr[512]; + const char *group = "share-group-resub-replace"; + const char *a = "0154-share-resub-A"; + const char *b = "0154-share-resub-B"; + const char *c = "0154-share-resub-C"; + const char *d = "0154-share-resub-D"; + + test_create_topic_wait_exists(NULL, a, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, b, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, c, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, d, 1, -1, 30 * 1000); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_t *rk = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(rk, "%s", errstr); + + rd_kafka_topic_partition_list_t *subs1 = + rd_kafka_topic_partition_list_new(2); + rd_kafka_topic_partition_list_add(subs1, a, RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs1, b, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(rk, subs1), "subscribe A,B failed"); + rd_kafka_topic_partition_list_destroy(subs1); + + rd_kafka_topic_partition_list_t *cur = NULL; + rd_kafka_subscription(rk, &cur); + TEST_ASSERT(cur && cur->cnt == 2, "expected 2 after first subscribe"); + rd_kafka_topic_partition_list_destroy(cur); + + rd_kafka_topic_partition_list_t *subs2 = + rd_kafka_topic_partition_list_new(2); + rd_kafka_topic_partition_list_add(subs2, c, RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs2, d, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(rk, subs2), "resubscribe C,D failed"); + rd_kafka_topic_partition_list_destroy(subs2); + + cur = NULL; + rd_kafka_subscription(rk, &cur); + TEST_ASSERT(cur && cur->cnt == 2, "expected 2 after resubscribe"); + for (int i = 0; i < cur->cnt; i++) { + const char *tn = cur->elems[i].topic; + TEST_ASSERT(strcmp(tn, a) && strcmp(tn, b), + "old topic %s still present", tn); + } + rd_kafka_topic_partition_list_destroy(cur); + + rd_kafka_consumer_close(rk); + rd_kafka_destroy(rk); +} + +/** + * Duplicate subscribe call with same list (idempotence). + */ +static void test_duplicate_subscribe_idempotent(void) { + char errstr[512]; + const char *group = "share-group-dup-sub"; + const char *t1 = "0154-share-dup-sub-1"; + const char *t2 = "0154-share-dup-sub-2"; + + test_create_topic_wait_exists(NULL, t1, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, t2, 1, -1, 30 * 1000); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_t *rk = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(rk, "%s", errstr); + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(2); + rd_kafka_topic_partition_list_add(subs, t1, RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs, t2, RD_KAFKA_PARTITION_UA); + + TEST_ASSERT(!rd_kafka_subscribe(rk, subs), "first subscribe failed"); + TEST_ASSERT(!rd_kafka_subscribe(rk, subs), + "duplicate subscribe failed"); + + rd_kafka_topic_partition_list_t *cur = NULL; + rd_kafka_subscription(rk, &cur); + TEST_ASSERT(cur && cur->cnt == 2, + "expected exactly 2 after duplicate subscribe"); + rd_kafka_topic_partition_list_destroy(cur); + + rd_kafka_topic_partition_list_destroy(subs); + rd_kafka_consumer_close(rk); + rd_kafka_destroy(rk); +} + +/** + * Subscribe to non-existent topic, then create it, produce, consume. + */ +static void test_subscribe_nonexistent_then_create(void) { + char errstr[512]; + const char *group = "share-group-sub-nonexist"; + const char *topic = test_mk_topic_name("0154-share-nonexist-topic", 1); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_t *rk = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(rk, "%s", errstr); + + const char *confs_set_group[] = {"share.auto.offset.reset", "SET", + "earliest"}; + test_IncrementalAlterConfigs_simple(rk, RD_KAFKA_RESOURCE_GROUP, group, + confs_set_group, 1); + + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subs, topic, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(rk, subs), + "subscribe non-existent failed"); + rd_kafka_topic_partition_list_destroy(subs); + + /* Confirm subscription shows the topic */ + rd_kafka_topic_partition_list_t *cur = NULL; + rd_kafka_subscription(rk, &cur); + TEST_ASSERT(cur && cur->cnt == 1, "expected 1 subscription"); + rd_kafka_topic_partition_list_destroy(cur); + + /* Now create topic and produce */ + test_create_topic_wait_exists(NULL, topic, 1, -1, 30 * 1000); + test_produce_msgs_easy(topic, 0, 0, 5); + + rd_kafka_message_t *batch[10]; + int got = 0; + int attempts = 10; + while (got < 5 && attempts-- > 0) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(rk, 2000, batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t i = 0; i < rcvd; i++) { + if (!batch[i]->err) + got++; + rd_kafka_message_destroy(batch[i]); + } + } + TEST_ASSERT(got == 5, + "expected 5 messages after topic creation, got %d", got); + + test_delete_topic(rk, topic); + rd_kafka_consumer_close(rk); + rd_kafka_destroy(rk); +} + +/** + * Unsubscribe then immediate subscribe to new topics: ensure old topics gone, + * only new consumed. + */ +static void test_unsubscribe_then_subscribe_new_topics(void) { + char errstr[512]; + const char *group = "share-group-unsub-resub"; + const char *old1 = "0154-share-old-1"; + const char *old2 = "0154-share-old-2"; + const char *new1 = "0154-share-new-1"; + const char *new2 = "0154-share-new-2"; + + test_create_topic_wait_exists(NULL, old1, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, old2, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, new1, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, new2, 1, -1, 30 * 1000); + + test_produce_msgs_easy(old1, 0, 0, 3); + test_produce_msgs_easy(old2, 0, 0, 3); + test_produce_msgs_easy(new1, 0, 0, 4); + test_produce_msgs_easy(new2, 0, 0, 4); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_t *rk = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(rk, "%s", errstr); + + const char *confs_set_group[] = {"share.auto.offset.reset", "SET", + "earliest"}; + test_IncrementalAlterConfigs_simple(rk, RD_KAFKA_RESOURCE_GROUP, group, + confs_set_group, 1); + + rd_kafka_topic_partition_list_t *subs_old = + rd_kafka_topic_partition_list_new(2); + rd_kafka_topic_partition_list_add(subs_old, old1, + RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs_old, old2, + RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(rk, subs_old), "subscribe old failed"); + rd_kafka_topic_partition_list_destroy(subs_old); + + /* Unsubscribe immediately */ + TEST_ASSERT(!rd_kafka_unsubscribe(rk), "unsubscribe failed"); + + /* Subscribe new topics */ + rd_kafka_topic_partition_list_t *subs_new = + rd_kafka_topic_partition_list_new(2); + rd_kafka_topic_partition_list_add(subs_new, new1, + RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(subs_new, new2, + RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(rk, subs_new), "subscribe new failed"); + rd_kafka_topic_partition_list_destroy(subs_new); + + /* Consume; ensure only new topics appear */ + rd_kafka_message_t *batch[50]; + int got_new = 0; + int attempts = 10; + while (got_new < 8 && attempts-- > 0) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(rk, 2000, batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t i = 0; i < rcvd; i++) { + if (!batch[i]->err) { + const char *tn = + rd_kafka_topic_name(batch[i]->rkt); + TEST_ASSERT( + strcmp(tn, old1) && strcmp(tn, old2), + "received message from old topic %s", tn); + if (!strcmp(tn, new1) || !strcmp(tn, new2)) + got_new++; + } + rd_kafka_message_destroy(batch[i]); + } + } + TEST_ASSERT(got_new == 8, "expected 8 new-topic msgs, got %d", got_new); + + test_delete_topic(rk, old1); + test_delete_topic(rk, old2); + test_delete_topic(rk, new1); + test_delete_topic(rk, new2); + + rd_kafka_consumer_close(rk); + rd_kafka_destroy(rk); +} + +/** + * Re-subscribe while messages exist: + * Consume some from A, then resubscribe to B only; ensure no A messages + * afterwards. + */ +static void test_resubscribe_switch_topics(void) { + char errstr[512]; + const char *group = "share-group-switch"; + const char *topicA = "0154-share-switch-A-resubscribe"; + const char *topicB = "0154-share-switch-B-resubscribe"; + const int msgsA_initial = 5; + const int msgsA_extra = 3; + const int msgsB_initial = 7; + const int msgsB_extra = 4; + + test_create_topic_wait_exists(NULL, topicA, 1, -1, 30 * 1000); + test_create_topic_wait_exists(NULL, topicB, 1, -1, 30 * 1000); + test_produce_msgs_easy(topicA, 0, 0, msgsA_initial); + test_produce_msgs_easy(topicB, 0, 0, msgsB_initial); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + rd_kafka_t *rk = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(rk, "%s", errstr); + + const char *grp_conf[] = {"share.auto.offset.reset", "SET", "earliest"}; + test_IncrementalAlterConfigs_simple(rk, RD_KAFKA_RESOURCE_GROUP, group, + grp_conf, 1); + + rd_kafka_topic_partition_list_t *subsA = + rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subsA, topicA, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(rk, subsA), "subscribe A failed"); + rd_kafka_topic_partition_list_destroy(subsA); + + int gotA = 0; + int attempts = 10; + rd_kafka_message_t *batch[128]; + while (gotA < msgsA_initial && attempts-- > 0) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(rk, 2000, batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t i = 0; i < rcvd; i++) { + if (!batch[i]->err && + !strcmp(rd_kafka_topic_name(batch[i]->rkt), topicA)) + gotA++; + rd_kafka_message_destroy(batch[i]); + } + } + TEST_ASSERT(gotA > 0, + "did not consume any messages from A before resubscribe"); + + /* Add extra messages to A that should not be seen after switching */ + test_produce_msgs_easy(topicA, 0, 0, msgsA_extra); + + /* Resubscribe to B only */ + rd_kafka_topic_partition_list_t *subsB = + rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subsB, topicB, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(rk, subsB), "resubscribe B failed"); + rd_kafka_topic_partition_list_destroy(subsB); + + /* Produce extra B after resubscribe */ + test_produce_msgs_easy(topicB, 0, 0, msgsB_extra); + + int wantB = msgsB_initial + msgsB_extra; + int gotB = 0; + attempts = 25; + + while (gotB < wantB && attempts-- > 0) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(rk, 3000, batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t i = 0; i < rcvd; i++) { + if (!batch[i]->err) { + const char *tn = + rd_kafka_topic_name(batch[i]->rkt); + TEST_ASSERT(!strcmp(tn, topicB), + "received message from old topic " + "%s after resubscribe", + tn); + gotB++; + } + rd_kafka_message_destroy(batch[i]); + } + } + + TEST_ASSERT(gotB == wantB, "expected %d B messages, got %d", wantB, + gotB); + + test_delete_topic(rk, topicA); + test_delete_topic(rk, topicB); + rd_kafka_consumer_close(rk); + rd_kafka_destroy(rk); +} + + +/** + * @brief Test that polling without subscription fails + */ +static void test_poll_no_subscribe_fails(void) { + char errstr[512]; + rd_kafka_conf_t *cons_conf; + rd_kafka_t *consumer; + char *group = "share-group-no-subscribe"; + + TEST_SAY("=== Testing poll without subscription fails ===\n"); + + /* Create share consumer */ + test_conf_init(&cons_conf, NULL, 60); + rd_kafka_conf_set(cons_conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + + consumer = + rd_kafka_new(RD_KAFKA_CONSUMER, cons_conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "Failed to create consumer: %s", errstr); + + /* Try to poll without subscribing - should fail or return timeout */ + TEST_SAY("Attempting to poll without subscription\n"); + rd_kafka_message_t **msgs = malloc(sizeof(rd_kafka_message_t *) * 10); + size_t rcvd_msgs = 0; + + rd_kafka_error_t *error = + rd_kafka_share_consume_batch(consumer, 2000, msgs, &rcvd_msgs); + + /** + * TODO KIP-932: Uncomment once polling before any subscription is + * properly handled + */ + // TEST_ASSERT(error, "Expected poll to fail after unsubscribe, but it + // succeeded"); + + free(msgs); + rd_kafka_destroy(consumer); +} + +/** + * @brief Test subscribe and poll with no records available + */ +static void test_subscribe_and_poll_no_records(void) { + char errstr[512]; + rd_kafka_conf_t *cons_conf; + rd_kafka_t *consumer; + rd_kafka_topic_partition_list_t *topics; + const char *topic = "0154-share-empty-records"; + const char *group = "share-group-empty"; + + TEST_SAY("=== Testing subscribe and poll with no records ===\n"); + + /* Create empty topic (no messages produced) */ + test_create_topic_wait_exists(NULL, topic, 1, -1, 60 * 1000); + TEST_SAY("Created empty topic: %s\n", topic); + + /* Create share consumer */ + test_conf_init(&cons_conf, NULL, 60); + rd_kafka_conf_set(cons_conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + + consumer = + rd_kafka_new(RD_KAFKA_CONSUMER, cons_conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "Failed to create consumer: %s", errstr); + + /* Subscribe to empty topic */ + topics = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(topics, topic, RD_KAFKA_PARTITION_UA); + rd_kafka_subscribe(consumer, topics); + rd_kafka_topic_partition_list_destroy(topics); + + TEST_SAY("Subscribed to empty topic, polling for messages\n"); + + /* Poll for messages - should get none */ + rd_kafka_message_t **msgs = malloc(sizeof(rd_kafka_message_t *) * 10); + size_t rcvd_msgs = 0; + + rd_kafka_error_t *error = + rd_kafka_share_consume_batch(consumer, 5000, msgs, &rcvd_msgs); + + TEST_ASSERT(rcvd_msgs == 0, + "Should not receive messages from empty topic"); + TEST_SAY("✓ No messages received from empty topic (expected)\n"); + + test_delete_topic(consumer, topic); + + free(msgs); + rd_kafka_destroy(consumer); +} + +/** + * @brief Test subscribe, poll, then unsubscribe + */ +static void test_subscribe_poll_unsubscribe(void) { + char errstr[512]; + rd_kafka_conf_t *cons_conf; + rd_kafka_t *consumer; + rd_kafka_topic_partition_list_t *topics; + const char *topic = "0154-share-unsub"; + const char *group = "share-group-unsub"; + const int msg_count = 5; + + TEST_SAY("=== Testing subscribe, poll, then unsubscribe ===\n"); + + /* Create topic and produce messages */ + test_create_topic_wait_exists(NULL, topic, 1, -1, 60 * 1000); + test_produce_msgs_easy(topic, 0, 0, msg_count); + TEST_SAY("Produced %d messages\n", msg_count); + + /* Create share consumer */ + test_conf_init(&cons_conf, NULL, 60); + rd_kafka_conf_set(cons_conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + + consumer = + rd_kafka_new(RD_KAFKA_CONSUMER, cons_conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "Failed to create consumer: %s", errstr); + + /* Subscribe to topic */ + topics = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(topics, topic, RD_KAFKA_PARTITION_UA); + rd_kafka_subscribe(consumer, topics); + rd_kafka_topic_partition_list_destroy(topics); + + TEST_SAY("Subscribed to topic, consuming messages\n"); + + /* Poll for some messages */ + rd_kafka_message_t **msgs = malloc(sizeof(rd_kafka_message_t *) * 10); + size_t rcvd_msgs = 0; + int consumed_count = 0; + + rd_kafka_error_t *error = + rd_kafka_share_consume_batch(consumer, 10000, msgs, &rcvd_msgs); + + if (!error && rcvd_msgs > 0) { + for (int i = 0; i < (int)rcvd_msgs; i++) { + if (!msgs[i]->err) { + consumed_count++; + } + rd_kafka_message_destroy(msgs[i]); + } + TEST_SAY("Consumed %d messages before unsubscribe\n", + consumed_count); + } else if (error) { + rd_kafka_error_destroy(error); + } + + /* Unsubscribe from all topics */ + TEST_SAY("Unsubscribing from all topics\n"); + rd_kafka_resp_err_t err = rd_kafka_unsubscribe(consumer); + TEST_ASSERT(!err, "Failed to unsubscribe: %s", rd_kafka_err2str(err)); + + /* Try to poll after unsubscribe - should fail or get no messages */ + TEST_SAY("Attempting to poll after unsubscribe\n"); + rcvd_msgs = 0; + error = rd_kafka_share_consume_batch(consumer, 2000, msgs, &rcvd_msgs); + + /** + * TODO KIP-932: Uncomment once polling before any subscription is + * properly handled + */ + // TEST_ASSERT(error, "Expected poll to fail after unsubscribe, but it + // succeeded"); + + test_delete_topic(consumer, topic); + + free(msgs); + rd_kafka_destroy(consumer); +} + +/** + * @brief Test subscribe, poll, then subscribe to different topic + */ +static void test_subscribe_poll_subscribe(void) { + char errstr[512]; + rd_kafka_conf_t *cons_conf; + rd_kafka_t *consumer; + rd_kafka_topic_partition_list_t *topics; + char *topic1 = "test-topic-0154-share-sub1"; + char *topic2 = "test-topic-0154-share-sub2"; + char *group = "share-group-resub"; + const int msg_count = 3; + + TEST_SAY("=== Testing subscribe, poll, then resubscribe ===\n"); + + /* Create topics and produce messages */ + test_create_topic_wait_exists(NULL, topic1, 1, -1, 60 * 1000); + test_create_topic_wait_exists(NULL, topic2, 1, -1, 60 * 1000); + + test_produce_msgs_easy(topic1, 0, 0, msg_count); + test_produce_msgs_easy(topic2, 0, 0, msg_count); + TEST_SAY("Produced %d messages to each topic\n", msg_count); + + /* Create share consumer */ + test_conf_init(&cons_conf, NULL, 60); + rd_kafka_conf_set(cons_conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + + consumer = + rd_kafka_new(RD_KAFKA_CONSUMER, cons_conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "Failed to create consumer: %s", errstr); + + /* Subscribe to first topic */ + topics = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(topics, topic1, + RD_KAFKA_PARTITION_UA); + rd_kafka_subscribe(consumer, topics); + rd_kafka_topic_partition_list_destroy(topics); + + TEST_SAY("Subscribed to first topic: %s\n", topic1); + + /* Poll from first topic */ + rd_kafka_message_t **msgs = malloc(sizeof(rd_kafka_message_t *) * 10); + size_t rcvd_msgs = 0; + int topic1_count = 0; + + rd_kafka_error_t *error = + rd_kafka_share_consume_batch(consumer, 10000, msgs, &rcvd_msgs); + + if (!error && rcvd_msgs > 0) { + for (int i = 0; i < (int)rcvd_msgs; i++) { + if (!msgs[i]->err) { + topic1_count++; + } + rd_kafka_message_destroy(msgs[i]); + } + TEST_SAY("Consumed %d messages from topic1\n", topic1_count); + } else if (error) { + rd_kafka_error_destroy(error); + } + + /* Subscribe to second topic */ + TEST_SAY("Resubscribing to second topic: %s\n", topic2); + topics = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(topics, topic2, + RD_KAFKA_PARTITION_UA); + rd_kafka_subscribe(consumer, topics); + rd_kafka_topic_partition_list_destroy(topics); + + /* Poll from second topic */ + rcvd_msgs = 0; + int topic2_count = 0; + + error = rd_kafka_share_consume_batch(consumer, 10000, msgs, &rcvd_msgs); + + if (!error && rcvd_msgs > 0) { + for (int i = 0; i < (int)rcvd_msgs; i++) { + if (!msgs[i]->err) { + topic2_count++; + } + rd_kafka_message_destroy(msgs[i]); + } + TEST_SAY("Consumed %d messages from topic2\n", topic2_count); + } else if (error) { + rd_kafka_error_destroy(error); + } + + TEST_SAY( + "✓ Successfully resubscribed and consumed from different topics\n"); + + test_delete_topic(consumer, topic1); + test_delete_topic(consumer, topic2); + + free(msgs); + rd_kafka_destroy(consumer); +} + +/** + * @brief Test subscribe, unsubscribe, then poll fails + */ +static void test_subscribe_unsubscribe_poll_fails(void) { + char errstr[512]; + rd_kafka_conf_t *cons_conf; + rd_kafka_t *consumer; + rd_kafka_topic_partition_list_t *topics; + const char *topic = "0154-share-unsub-fail"; + const char *group = "share-group-unsub-fail"; + + TEST_SAY("=== Testing subscribe, unsubscribe, then poll fails ===\n"); + + /* Create topic */ + test_create_topic_wait_exists(NULL, topic, 1, -1, 60 * 1000); + test_produce_msgs_easy(topic, 0, 0, 3); + + /* Create share consumer */ + test_conf_init(&cons_conf, NULL, 60); + rd_kafka_conf_set(cons_conf, "share.consumer", "true", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.protocol", "consumer", errstr, + sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "enable.auto.commit", "false", errstr, + sizeof(errstr)); + + consumer = + rd_kafka_new(RD_KAFKA_CONSUMER, cons_conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "Failed to create consumer: %s", errstr); + + /* Subscribe to topic */ + topics = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(topics, topic, RD_KAFKA_PARTITION_UA); + rd_kafka_subscribe(consumer, topics); + rd_kafka_topic_partition_list_destroy(topics); + + TEST_SAY("Subscribed to topic: %s\n", topic); + + /* Immediately unsubscribe */ + TEST_SAY("Unsubscribing immediately\n"); + rd_kafka_resp_err_t err = rd_kafka_unsubscribe(consumer); + TEST_ASSERT(!err, "Failed to unsubscribe: %s", rd_kafka_err2str(err)); + + /* Try to poll - should fail */ + TEST_SAY("Attempting to poll after unsubscribe\n"); + rd_kafka_message_t **msgs = malloc(sizeof(rd_kafka_message_t *) * 10); + size_t rcvd_msgs = 0; + + rd_kafka_error_t *error = + rd_kafka_share_consume_batch(consumer, 2000, msgs, &rcvd_msgs); + + /** + * TODO KIP-932: Uncomment once polling before any subscription is + * properly handled + */ + // TEST_ASSERT(error, "Expected poll to fail after unsubscribe, but it + // succeeded"); + + test_delete_topic(consumer, topic); + + free(msgs); + rd_kafka_destroy(consumer); +} + +int main_0170_share_consumer_subscription(int argc, char **argv) { + test_subscription_introspection(); + test_unsubscribe_idempotence(); + test_resubscribe_replaces_set(); + test_duplicate_subscribe_idempotent(); + test_subscribe_nonexistent_then_create(); + test_unsubscribe_then_subscribe_new_topics(); + test_resubscribe_switch_topics(); + test_poll_no_subscribe_fails(); + test_subscribe_and_poll_no_records(); + test_subscribe_poll_unsubscribe(); + test_subscribe_poll_subscribe(); + test_subscribe_unsubscribe_poll_fails(); + return 0; +} \ No newline at end of file diff --git a/tests/0171-share_consumer_consume.c b/tests/0171-share_consumer_consume.c new file mode 100644 index 0000000000..d9bbc7ef19 --- /dev/null +++ b/tests/0171-share_consumer_consume.c @@ -0,0 +1,773 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2025, Confluent Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +#include "test.h" + + +/** + * @brief Test producing and consuming 10 messages + */ +static void test_share_consumer_messages(void) { + char errstr[512]; + rd_kafka_conf_t *cons_conf; + rd_kafka_t *consumer; + rd_kafka_topic_partition_list_t *topics; + const char *topic = "0154-share-test"; + char *group = "share-group-10msg"; + const int msg_count = 10; + int consumed_count = 0; + int attempts = 10; // Number of attempts to poll so the test doesn't run indefinitely + + const char *confs_set_group[] = {"share.auto.offset.reset", + "SET", "earliest"}; + + TEST_SAY("=== Testing share consumer with 10 messages ===\n"); + + /* Create topic */ + test_create_topic_wait_exists(NULL, topic, 1, -1, 60 * 1000); + rd_sleep(5); + + test_produce_msgs_easy(topic, 0, 0, msg_count); + TEST_SAY("Successfully produced %d messages\n", msg_count); + + /* Create share consumer */ + TEST_SAY("Creating share consumer for group %s\n", group); + test_conf_init(&cons_conf, NULL, 60); + + rd_kafka_conf_set(cons_conf, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "enable.auto.commit", "false", errstr, sizeof(errstr)); + + consumer = rd_kafka_new(RD_KAFKA_CONSUMER, cons_conf, errstr, sizeof(errstr)); + if (!consumer) { + TEST_FAIL("Failed to create share consumer: %s", errstr); + } + + /* Subscribe to topic */ + topics = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(topics, topic, RD_KAFKA_PARTITION_UA); + + TEST_SAY("Subscribing to topic %s\n", topic); + rd_kafka_subscribe(consumer, topics); + rd_kafka_topic_partition_list_destroy(topics); + + test_IncrementalAlterConfigs_simple(consumer, RD_KAFKA_RESOURCE_GROUP, group, confs_set_group, 1); + + /* Allocate message array for batch consumption */ + rd_kafka_message_t **rkmessages = malloc(sizeof(rd_kafka_message_t *) * 100); + + /* Consume messages until we get all 10 */ + while (consumed_count < msg_count && attempts > 0) { + size_t rcvd_msgs = 0; + rd_kafka_error_t *error; + + error = rd_kafka_share_consume_batch(consumer, 5000, rkmessages, &rcvd_msgs); + + if (error) { + TEST_SAY("Consume error: %s\n", rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + attempts--; + continue; + } + + TEST_SAY("Received %zu messages in batch\n", rcvd_msgs); + + /* Process each message in the batch */ + for (int i = 0; i < (int)rcvd_msgs; i++) { + rd_kafka_message_t *rkm = rkmessages[i]; + + if (rkm->err) { + TEST_SAY("Message error: %s\n", rd_kafka_message_errstr(rkm)); + rd_kafka_message_destroy(rkm); + continue; + } + + /* Count valid messages only */ + consumed_count++; + TEST_SAY("Consumed message %d/%d\n", consumed_count, msg_count); + + /* Clean up message */ + rd_kafka_message_destroy(rkm); + } + + /* Break if we've consumed all expected messages */ + if (consumed_count >= msg_count) { + TEST_SAY("Consumed all %d messages, stopping\n", msg_count); + break; + } + + attempts--; + } + + free(rkmessages); + + /* Verify we got exactly the expected number of messages */ + TEST_ASSERT(consumed_count == msg_count, + "Expected to consume %d messages, but consumed %d", + msg_count, consumed_count); + + TEST_SAY("✓ Successfully consumed exactly %d messages\n", consumed_count); + + test_delete_topic(consumer, topic); + + /* Clean up */ + rd_kafka_destroy(consumer); +} + +/** + * @brief Test subscribing to multiple topics + */ +static void test_share_consumer_multiple_topics(void) { + char errstr[512]; + rd_kafka_conf_t *cons_conf; + rd_kafka_t *consumer; + rd_kafka_topic_partition_list_t *topics; + char *topic1 = "0154-share-topic-multi-1"; + char *topic2 = "0154-share-topic-multi-2"; + char *group = "share-group-multitopic"; + const int msgs_per_topic = 5; + int consumed_count = 0; + + const char *confs_set_group[] = {"share.auto.offset.reset", + "SET", "earliest"}; + + TEST_SAY("=== Testing share consumer with multiple topics ===\n"); + + /* Create topics and produce messages */ + test_create_topic_wait_exists(NULL, topic1, 1, -1, 60 * 1000); + test_create_topic_wait_exists(NULL, topic2, 1, -1, 60 * 1000); + + test_produce_msgs_easy(topic1, 0, 0, msgs_per_topic); + test_produce_msgs_easy(topic2, 0, 0, msgs_per_topic); + TEST_SAY("Produced %d messages to each topic\n", msgs_per_topic); + + /* Create share consumer */ + test_conf_init(&cons_conf, NULL, 60); + rd_kafka_conf_set(cons_conf, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(cons_conf, "enable.auto.commit", "false", errstr, sizeof(errstr)); + + consumer = rd_kafka_new(RD_KAFKA_CONSUMER, cons_conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "Failed to create consumer: %s", errstr); + + test_IncrementalAlterConfigs_simple(consumer, RD_KAFKA_RESOURCE_GROUP, group, confs_set_group, 1); + + /* Subscribe to both topics */ + topics = rd_kafka_topic_partition_list_new(2); + rd_kafka_topic_partition_list_add(topics, topic1, RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(topics, topic2, RD_KAFKA_PARTITION_UA); + + TEST_SAY("Subscribing to topics: %s, %s\n", topic1, topic2); + rd_kafka_subscribe(consumer, topics); + rd_kafka_topic_partition_list_destroy(topics); + + /* Consume messages from both topics */ + rd_kafka_message_t **rkmessages = malloc(sizeof(rd_kafka_message_t *) * 500); + int attempts = 10; // Number of attempts to poll so the test doesn't run indefinitely + + while (consumed_count < (msgs_per_topic * 2) && attempts > 0) { + size_t rcvd_msgs = 0; + rd_kafka_error_t *error; + int i; + + error = rd_kafka_share_consume_batch(consumer, 3000, rkmessages, &rcvd_msgs); + + if (error) { + TEST_SAY("Consume error: %s\n", rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + attempts--; + continue; + } + + for (i = 0; i < (int)rcvd_msgs; i++) { + rd_kafka_message_t *rkm = rkmessages[i]; + if (!rkm->err) { + consumed_count++; + TEST_SAY("Consumed from topic %s: %d/%d total\n", + rd_kafka_topic_name(rkm->rkt), consumed_count, msgs_per_topic * 2); + } + rd_kafka_message_destroy(rkm); + } + attempts--; + } + + free(rkmessages); + + TEST_ASSERT(consumed_count == (msgs_per_topic * 2), + "Expected %d messages from both topics, got %d", + msgs_per_topic * 2, consumed_count); + + TEST_SAY("✓ Successfully consumed from multiple topics: %d messages\n", consumed_count); + + test_delete_topic(consumer, topic1); + test_delete_topic(consumer, topic2); + + /* Clean up */ + rd_kafka_consumer_close(consumer); + rd_kafka_destroy(consumer); +} + +/** + * @brief Test multiple share consumers on the same topic. + * Verifies that messages are divided (each delivered once overall). + */ +static void test_share_consumer_multi_members_same_topic(void) { + char errstr[512]; + const char *group = "share-group-multi-member"; + char *topic = "0154-share-multi-member"; + const int total_msgs = 1000; + int consumed_total = 0; + int c1_count = 0; + int c2_count = 0; + int attempts = 15; + const char *group_conf[] = {"share.auto.offset.reset","SET","earliest"}; + + TEST_SAY("=== Testing multiple share consumers on same topic ===\n"); + + /* Create topic and produce messages */ + test_create_topic_wait_exists(NULL, topic, 1, -1, 60 * 1000); + test_produce_msgs_easy(topic, 0, 0, total_msgs); + TEST_SAY("Produced %d messages to %s\n", total_msgs, topic); + + /* Common subscription list */ + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subs, topic, RD_KAFKA_PARTITION_UA); + + /* Create first consumer */ + rd_kafka_conf_t *conf1; + test_conf_init(&conf1, NULL, 60); + rd_kafka_conf_set(conf1, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf1, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf1, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf1, "enable.auto.commit", "false", errstr, sizeof(errstr)); + rd_kafka_t *c1 = rd_kafka_new(RD_KAFKA_CONSUMER, conf1, errstr, sizeof(errstr)); + TEST_ASSERT(c1, "Failed to create consumer1: %s", errstr); + + /* Create second consumer */ + rd_kafka_conf_t *conf2; + test_conf_init(&conf2, NULL, 60); + rd_kafka_conf_set(conf2, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf2, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf2, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf2, "enable.auto.commit", "false", errstr, sizeof(errstr)); + rd_kafka_t *c2 = rd_kafka_new(RD_KAFKA_CONSUMER, conf2, errstr, sizeof(errstr)); + TEST_ASSERT(c2, "Failed to create consumer2: %s", errstr); + + /* Set group config (offset reset) to earliest */ + test_IncrementalAlterConfigs_simple(c1, RD_KAFKA_RESOURCE_GROUP, + group, group_conf, 1); + + /* Subscribe both */ + rd_kafka_subscribe(c1, subs); + rd_kafka_subscribe(c2, subs); + rd_kafka_topic_partition_list_destroy(subs); + + /* Poll loop: alternate polling both consumers */ + rd_kafka_message_t *batch[500]; + + while (consumed_total < total_msgs && attempts-- > 0) { + size_t rcvd1 = 0, rcvd2 = 0; + rd_kafka_error_t *err1 = + rd_kafka_share_consume_batch(c1, 2000, batch, &rcvd1); + if (!err1) { + for (size_t i = 0; i < rcvd1; i++) { + if (!batch[i]->err) { + c1_count++; + consumed_total++; + } + rd_kafka_message_destroy(batch[i]); + } + } else { + rd_kafka_error_destroy(err1); + } + + if (consumed_total >= total_msgs) + break; + + rd_kafka_error_t *err2 = + rd_kafka_share_consume_batch(c2, 2000, batch, &rcvd2); + if (!err2) { + for (size_t i = 0; i < rcvd2; i++) { + if (!batch[i]->err) { + c2_count++; + consumed_total++; + } + rd_kafka_message_destroy(batch[i]); + } + } else { + rd_kafka_error_destroy(err2); + } + + TEST_SAY("Progress: total=%d/%d (c1=%d, c2=%d)\n", + consumed_total, total_msgs, c1_count, c2_count); + } + + TEST_ASSERT(consumed_total == total_msgs, + "Expected %d total messages, got %d", total_msgs, + consumed_total); + + TEST_SAY("✓ Multi-member share consumption complete: total=%d " + "(c1=%d, c2=%d)\n", + consumed_total, c1_count, c2_count); + + test_delete_topic(c1, topic); + + rd_kafka_consumer_close(c1); + rd_kafka_consumer_close(c2); + rd_kafka_destroy(c1); + rd_kafka_destroy(c2); +} + +/** + * Single share consumer, one topic with multiple partitions. + */ +static void test_share_single_consumer_multi_partitions_one_topic(void) { + char errstr[512]; + const char *group = "share-group-single-one-topic-mparts"; + const char *topic = "0154-share-one-topic-mparts"; + const int partition_cnt = 3; + const int msgs_per_partition = 500; + const int total_msgs = partition_cnt * msgs_per_partition; + int consumed = 0; + int attempts = 30; + const char *grp_conf[] = {"share.auto.offset.reset","SET","earliest"}; + + TEST_SAY("=== Single consumer, one topic (%d partitions) ===\n", + partition_cnt); + + test_create_topic_wait_exists(NULL, topic, partition_cnt, -1, + 60 * 1000); + + for (int p = 0; p < partition_cnt; p++) + test_produce_msgs_easy(topic, p, p, + msgs_per_partition); + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, sizeof(errstr)); + + rd_kafka_t *consumer = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "create failed: %s", errstr); + + test_IncrementalAlterConfigs_simple(consumer, RD_KAFKA_RESOURCE_GROUP, + group, grp_conf, 1); + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subs, topic, RD_KAFKA_PARTITION_UA); + rd_kafka_subscribe(consumer, subs); + rd_kafka_topic_partition_list_destroy(subs); + + rd_kafka_message_t *batch[500]; + + while (consumed < total_msgs && attempts-- > 0) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(consumer, 3000, batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t i = 0; i < rcvd; i++) { + if (!batch[i]->err) + consumed++; + rd_kafka_message_destroy(batch[i]); + } + TEST_SAY("Progress: %d/%d\n", consumed, total_msgs); + } + + TEST_ASSERT(consumed == total_msgs, + "Expected %d, got %d", total_msgs, consumed); + + TEST_SAY("✓ Consumed all %d messages across %d partitions\n", + consumed, partition_cnt); + + test_delete_topic(consumer, topic); + rd_kafka_consumer_close(consumer); + rd_kafka_destroy(consumer); +} + +/** + * Single share consumer, multiple topics each with multiple partitions. + */ +static void test_share_single_consumer_multi_partitions_multi_topics(void) { + char errstr[512]; + const char *group = "share-group-single-multi-topic-mparts"; + const int topic_cnt = 3; + const int partition_cnt = 2; + const int msgs_per_partition = 500; + char *topics[topic_cnt]; + int total_msgs = topic_cnt * partition_cnt * msgs_per_partition; + int consumed = 0; + int attempts = 40; + const char *grp_conf[] = {"share.auto.offset.reset","SET","earliest"}; + + TEST_SAY("=== Single consumer, %d topics x %d partitions ===\n", + topic_cnt, partition_cnt); + + for (int t = 0; t < topic_cnt; t++) { + topics[t] = rd_strdup(test_mk_topic_name("0154-share-multiT-mparts", 1)); + test_create_topic_wait_exists(NULL, topics[t], partition_cnt, + -1, 60 * 1000); + for (int p = 0; p < partition_cnt; p++) + test_produce_msgs_easy(topics[t], p, + p, + msgs_per_partition); + } + + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, sizeof(errstr)); + + rd_kafka_t *consumer = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "create failed: %s", errstr); + + test_IncrementalAlterConfigs_simple(consumer, RD_KAFKA_RESOURCE_GROUP, + group, grp_conf, 1); + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(topic_cnt); + for (int t = 0; t < topic_cnt; t++) + rd_kafka_topic_partition_list_add(subs, topics[t], + RD_KAFKA_PARTITION_UA); + rd_kafka_subscribe(consumer, subs); + rd_kafka_topic_partition_list_destroy(subs); + + rd_kafka_message_t *batch[500]; + + while (consumed < total_msgs && attempts-- > 0) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(consumer, 3000, batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t i = 0; i < rcvd; i++) { + if (!batch[i]->err) + consumed++; + rd_kafka_message_destroy(batch[i]); + } + TEST_SAY("Progress: %d/%d\n", consumed, total_msgs); + } + + TEST_ASSERT(consumed == total_msgs, + "Expected %d, got %d", total_msgs, consumed); + TEST_SAY("✓ Consumed all %d messages from %d topics\n", + consumed, topic_cnt); + + for (int t = 0; t < topic_cnt; t++) + test_delete_topic(consumer, topics[t]); + + rd_kafka_consumer_close(consumer); + rd_kafka_destroy(consumer); +} + +/** + * Multiple share consumers, one topic with multiple partitions. + */ +static void test_share_multi_consumers_multi_partitions_one_topic(void) { + char errstr[512]; + const char *group = "share-group-multi-cons-one-topic-mparts"; + const char *topic = "0154-share-cons-oneT-mparts"; + const int partition_cnt = 4; + const int msgs_per_partition = 500; + const int total_msgs = partition_cnt * msgs_per_partition; + int consumed_total = 0; + int c_counts[4] = {0}; + int attempts = 50; + const int consumer_cnt = 2; + rd_kafka_t *consumers[consumer_cnt]; + const char *grp_conf[] = {"share.auto.offset.reset","SET","earliest"}; + + TEST_SAY("=== %d consumers, one topic, %d partitions ===\n", + consumer_cnt, partition_cnt); + + test_create_topic_wait_exists(NULL, topic, partition_cnt, -1, + 60 * 1000); + for (int p = 0; p < partition_cnt; p++) + test_produce_msgs_easy(topic, p, p , + msgs_per_partition); + + for (int i = 0; i < consumer_cnt; i++) { + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, sizeof(errstr)); + consumers[i] = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumers[i], "create failed (%d): %s", i, errstr); + } + + test_IncrementalAlterConfigs_simple(consumers[0], RD_KAFKA_RESOURCE_GROUP, + group, grp_conf, 1); + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subs, topic, RD_KAFKA_PARTITION_UA); + for (int i = 0; i < consumer_cnt; i++) + rd_kafka_subscribe(consumers[i], subs); + rd_kafka_topic_partition_list_destroy(subs); + + rd_kafka_message_t *batch[500]; + + while (consumed_total < total_msgs && attempts-- > 0) { + for (int i = 0; i < consumer_cnt; i++) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(consumers[i], 1000, + batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t m = 0; m < rcvd; m++) { + if (!batch[m]->err) { + c_counts[i]++; + consumed_total++; + } + rd_kafka_message_destroy(batch[m]); + } + } + TEST_SAY("Progress: total=%d/%d c0=%d c1=%d\n", + consumed_total, total_msgs, + c_counts[0], c_counts[1]); + } + + TEST_ASSERT(consumed_total == total_msgs, + "Expected %d total, got %d", total_msgs, consumed_total); + + TEST_SAY("✓ All %d messages consumed by %d consumers " + "(dist: c0=%d c1=%d)\n", + consumed_total, consumer_cnt, c_counts[0], c_counts[1]); + + test_delete_topic(consumers[0], topic); + for (int i = 0; i < consumer_cnt; i++) { + rd_kafka_consumer_close(consumers[i]); + rd_kafka_destroy(consumers[i]); + } +} + +/** + * Multiple consumers, multiple topics each with multiple partitions. + */ +static void test_share_multi_consumers_multi_partitions_multi_topics(void) { + char errstr[512]; + const char *group = "share-group-multi-cons-multiT-mparts"; + const int topic_cnt = 2; + const int partition_cnt = 3; + const int msgs_per_partition = 500; + const int consumer_cnt = 3; + char *topics[topic_cnt]; + int total_msgs = topic_cnt * partition_cnt * msgs_per_partition; + int consumed_total = 0; + int per_cons[consumer_cnt]; + memset(per_cons, 0, sizeof(per_cons)); + int attempts = 80; + rd_kafka_t *consumers[consumer_cnt]; + const char *grp_conf[] = {"share.auto.offset.reset","SET","earliest"}; + + TEST_SAY("=== %d consumers, %d topics x %d partitions ===\n", + consumer_cnt, topic_cnt, partition_cnt); + + for (int t = 0; t < topic_cnt; t++) { + topics[t] = rd_strdup(test_mk_topic_name("0154-share-multiT", 1)); + test_create_topic_wait_exists(NULL, topics[t], partition_cnt, + -1, 60 * 1000); + for (int p = 0; p < partition_cnt; p++) + test_produce_msgs_easy(topics[t], p, + p, + msgs_per_partition); + } + + for (int i = 0; i < consumer_cnt; i++) { + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, sizeof(errstr)); + consumers[i] = + rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumers[i], "create failed (%d): %s", i, errstr); + } + + test_IncrementalAlterConfigs_simple(consumers[0], RD_KAFKA_RESOURCE_GROUP, + group, grp_conf, 1); + + rd_kafka_topic_partition_list_t *subs = + rd_kafka_topic_partition_list_new(topic_cnt); + for (int t = 0; t < topic_cnt; t++) + rd_kafka_topic_partition_list_add(subs, topics[t], + RD_KAFKA_PARTITION_UA); + for (int i = 0; i < consumer_cnt; i++) + rd_kafka_subscribe(consumers[i], subs); + rd_kafka_topic_partition_list_destroy(subs); + + rd_kafka_message_t *batch[500]; + + while (consumed_total < total_msgs && attempts-- > 0) { + for (int i = 0; i < consumer_cnt; i++) { + size_t rcvd = 0; + rd_kafka_error_t *err = + rd_kafka_share_consume_batch(consumers[i], 1000, + batch, &rcvd); + if (err) { + rd_kafka_error_destroy(err); + continue; + } + for (size_t m = 0; m < rcvd; m++) { + if (!batch[m]->err) { + per_cons[i]++; + consumed_total++; + } + rd_kafka_message_destroy(batch[m]); + } + } + TEST_SAY("Progress: total=%d/%d c0=%d c1=%d c2=%d\n", + consumed_total, total_msgs, + per_cons[0], per_cons[1], per_cons[2]); + } + + TEST_ASSERT(consumed_total == total_msgs, + "Expected %d total, got %d", total_msgs, consumed_total); + + TEST_SAY("✓ All %d messages consumed across %d consumers " + "(dist: c0=%d c1=%d c2=%d)\n", + consumed_total, consumer_cnt, + per_cons[0], per_cons[1], per_cons[2]); + + for (int t = 0; t < topic_cnt; t++) + test_delete_topic(consumers[0], topics[t]); + + for (int i = 0; i < consumer_cnt; i++) { + rd_kafka_consumer_close(consumers[i]); + rd_kafka_destroy(consumers[i]); + } +} + +static void test_batch_all_partitions_arrive_together(void) { + char errstr[512]; + const char *group = "share-group-batch-all"; + const char *topic = test_mk_topic_name("0154-share-batch-all", 1); + const int partition_cnt = 3; + const int msgs_per_partition = 5; + const int total_msgs = partition_cnt * msgs_per_partition; + + TEST_SAY("=== Expect single batch of %d msgs across %d partitions ===\n", + total_msgs, partition_cnt); + + /* Create topic with 3 partitions and produce 5 msgs per partition */ + test_create_topic_wait_exists(NULL, topic, partition_cnt, -1, 60 * 1000); + for (int p = 0; p < partition_cnt; p++) + test_produce_msgs_easy(topic, p, p, msgs_per_partition); + + /* Create share consumer */ + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 60); + rd_kafka_conf_set(conf, "share.consumer", "true", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "group.id", group, errstr, sizeof(errstr)); + rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr, sizeof(errstr)); + rd_kafka_t *consumer = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(consumer, "Failed to create consumer: %s", errstr); + + /* Read from beginning */ + const char *grp_conf[] = {"share.auto.offset.reset","SET","earliest"}; + test_IncrementalAlterConfigs_simple(consumer, RD_KAFKA_RESOURCE_GROUP, group, grp_conf, 1); + + /* Subscribe */ + rd_kafka_topic_partition_list_t *subs = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subs, topic, RD_KAFKA_PARTITION_UA); + TEST_ASSERT(!rd_kafka_subscribe(consumer, subs), "subscribe failed"); + rd_kafka_topic_partition_list_destroy(subs); + + /* Consume once, expect all 15 messages in single batch */ + rd_kafka_message_t *msgs[32]; + size_t rcvd = 0; + int counts[partition_cnt]; + memset(counts, 0, sizeof(counts)); + + rd_kafka_error_t *err = rd_kafka_share_consume_batch(consumer, 10000, msgs, &rcvd); + TEST_ASSERT(!err, "Consume error: %s", err ? rd_kafka_error_string(err) : ""); + if (err) rd_kafka_error_destroy(err); + + TEST_SAY("Received %zu messages in one batch\n", rcvd); + TEST_ASSERT(rcvd == (size_t)total_msgs, + "Expected %d messages in single batch, got %zu", + total_msgs, rcvd); + + /* Verify 5 per partition, destroy messages */ + for (size_t i = 0; i < rcvd; i++) { + rd_kafka_message_t *m = msgs[i]; + TEST_ASSERT(!m->err, "Message error: %s", rd_kafka_message_errstr(m)); + int p = m->partition; + TEST_ASSERT(p >= 0 && p < partition_cnt, "Unexpected partition %d", p); + counts[p]++; + rd_kafka_message_destroy(m); + } + for (int p = 0; p < partition_cnt; p++) { + TEST_ASSERT(counts[p] == msgs_per_partition, + "Partition %d expected %d msgs, got %d", + p, msgs_per_partition, counts[p]); + } + + TEST_SAY("✓ Single batch contained all %d msgs (5 per partition)\n", total_msgs); + + test_delete_topic(consumer, topic); + rd_kafka_consumer_close(consumer); + rd_kafka_destroy(consumer); +} + + +int main_0171_share_consumer_consume(int argc, char **argv) { + + test_share_consumer_messages(); + test_share_consumer_multiple_topics(); + test_share_consumer_multi_members_same_topic(); + test_share_single_consumer_multi_partitions_one_topic(); + test_share_single_consumer_multi_partitions_multi_topics(); + test_share_multi_consumers_multi_partitions_one_topic(); + test_share_multi_consumers_multi_partitions_multi_topics(); + /* Uncomment it after fixing the issue */ + //test_batch_all_partitions_arrive_together(); + return 0; +} diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index e509092873..ca5b598c4b 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -143,6 +143,8 @@ set( 0151-purge-brokers.c 0152-rebootstrap.c 0153-memberid.c + 0170-share_consumer_subscription.c + 0171-share_consumer_consume.c 8000-idle.cpp 8001-fetch_from_follower_mock_manual.c test.c diff --git a/tests/broker_version_tests.py b/tests/broker_version_tests.py index c451e02471..1a0de874b1 100755 --- a/tests/broker_version_tests.py +++ b/tests/broker_version_tests.py @@ -31,7 +31,7 @@ def test_it(version, deploy=True, conf={}, rdkconf={}, tests=None, """ cluster = LibrdkafkaTestCluster(version, conf, - num_brokers=int(conf.get('broker_cnt', 3)), + num_brokers=int(conf.get('broker_cnt', 1)), debug=debug, scenario=scenario, kraft=kraft) @@ -175,7 +175,7 @@ def handle_report(report, version, suite): '--brokers', dest='broker_cnt', type=int, - default=3, + default=1, help='Number of Kafka brokers') parser.add_argument('--ssl', dest='ssl', action='store_true', default=False, diff --git a/tests/interactive_broker_version.py b/tests/interactive_broker_version.py index acddc872fd..3f2ffe7aae 100755 --- a/tests/interactive_broker_version.py +++ b/tests/interactive_broker_version.py @@ -25,7 +25,7 @@ def version_as_number(version): def test_version(version, cmd=None, deploy=True, conf={}, debug=False, exec_cnt=1, - root_path='tmp', broker_cnt=3, scenario='default', + root_path='tmp', broker_cnt=1, scenario='default', kraft=False): """ @brief Create, deploy and start a Kafka cluster using Kafka \\p version diff --git a/tests/test.c b/tests/test.c index 86205dd5de..7646de63c5 100644 --- a/tests/test.c +++ b/tests/test.c @@ -270,6 +270,8 @@ _TEST_DECL(0150_telemetry_mock); _TEST_DECL(0151_purge_brokers_mock); _TEST_DECL(0152_rebootstrap_local); _TEST_DECL(0153_memberid); +_TEST_DECL(0170_share_consumer_subscription); +_TEST_DECL(0171_share_consumer_consume); /* Manual tests */ _TEST_DECL(8000_idle); @@ -536,6 +538,8 @@ struct test tests[] = { _TEST(0151_purge_brokers_mock, TEST_F_LOCAL), _TEST(0152_rebootstrap_local, TEST_F_LOCAL), _TEST(0153_memberid, 0, TEST_BRKVER(0, 4, 0, 0)), + _TEST(0170_share_consumer_subscription, 0, TEST_BRKVER(0, 4, 0, 0)), + _TEST(0171_share_consumer_consume, 0, TEST_BRKVER(0, 4, 0, 0)), /* Manual tests */ diff --git a/win32/tests/tests.vcxproj b/win32/tests/tests.vcxproj index c00bc84418..9c8f824a23 100644 --- a/win32/tests/tests.vcxproj +++ b/win32/tests/tests.vcxproj @@ -233,6 +233,8 @@ + +