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/.gitignore b/examples/.gitignore
index a2cd3d609a..cfc1739a50 100644
--- a/examples/.gitignore
+++ b/examples/.gitignore
@@ -22,4 +22,5 @@ alter_consumer_group_offsets
incremental_alter_configs
user_scram
list_offsets
-elect_leaders
\ No newline at end of file
+elect_leaders
+share_consumer
diff --git a/examples/Makefile b/examples/Makefile
index f8b6513540..e9d64e30d4 100644
--- a/examples/Makefile
+++ b/examples/Makefile
@@ -1,7 +1,7 @@
EXAMPLES ?= rdkafka_example rdkafka_performance rdkafka_example_cpp \
rdkafka_complex_consumer_example rdkafka_complex_consumer_example_cpp \
kafkatest_verifiable_client \
- producer consumer idempotent_producer transactions \
+ producer consumer share_consumer idempotent_producer transactions \
delete_records \
openssl_engine_example_cpp \
list_consumer_groups \
@@ -54,6 +54,10 @@ consumer: ../src/librdkafka.a consumer.c
$(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \
../src/librdkafka.a $(LIBS)
+share_consumer: ../src/librdkafka.a share_consumer.c
+ $(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \
+ ../src/librdkafka.a $(LIBS)
+
idempotent_producer: ../src/librdkafka.a idempotent_producer.c
$(CC) $(CPPFLAGS) $(CFLAGS) $@.c -o $@ $(LDFLAGS) \
../src/librdkafka.a $(LIBS)
diff --git a/examples/share_consumer.c b/examples/share_consumer.c
new file mode 100644
index 0000000000..73cbede97e
--- /dev/null
+++ b/examples/share_consumer.c
@@ -0,0 +1,298 @@
+/*
+ * librdkafka - Apache Kafka C library
+ *
+ * Copyright (c) 2019-2022, Magnus Edenhill
+ * 2023, 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.
+ */
+
+/**
+ * Simple high-level balanced Apache Kafka consumer
+ * using the Kafka driver from librdkafka
+ * (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
+ * is builtin from within the librdkafka source tree and thus differs. */
+// #include
+#include "rdkafka.h"
+
+/* ANSI color codes */
+#define ANSI_RED "\033[31m"
+#define ANSI_GREEN "\033[32m"
+#define ANSI_YELLOW "\033[33m"
+#define ANSI_CYAN "\033[36m"
+#define ANSI_RESET "\033[0m"
+
+#define TIME_BLOCK_MS(elapsed_var, expr) \
+ do { \
+ struct timespec __t0, __t1; \
+ if (clock_gettime(CLOCK_MONOTONIC, &__t0) != 0) \
+ perror("clock_gettime"); \
+ expr; \
+ if (clock_gettime(CLOCK_MONOTONIC, &__t1) != 0) \
+ perror("clock_gettime"); \
+ (elapsed_var) = (__t1.tv_sec - __t0.tv_sec) * 1000.0 + \
+ (__t1.tv_nsec - __t0.tv_nsec) / 1e6; \
+ } while (0)
+
+
+static volatile sig_atomic_t run = 1;
+
+/**
+ * @brief Signal termination of program
+ */
+static void stop(int sig) {
+ run = 0;
+}
+
+
+
+/**
+ * @returns 1 if all bytes are printable, else 0.
+ */
+static int is_printable(const char *buf, size_t size) {
+ size_t i;
+
+ for (i = 0; i < size; i++)
+ if (!isprint((int)buf[i]))
+ return 0;
+
+ return 1;
+}
+
+
+int main(int argc, char **argv) {
+ rd_kafka_share_t *rkshare; /* Consumer instance handle */
+ rd_kafka_conf_t *conf; /* Temporary configuration object */
+ rd_kafka_resp_err_t err; /* librdkafka API error code */
+ char errstr[512]; /* librdkafka API error reporting buffer */
+ const char *brokers; /* Argument: broker list */
+ const char *groupid; /* Argument: Consumer group id */
+ char **topics; /* Argument: list of topics to subscribe to */
+ int topic_cnt; /* Number of topics to subscribe to */
+ rd_kafka_topic_partition_list_t *subscription; /* Subscribed topics */
+ int i;
+
+ /*
+ * Argument validation
+ */
+ if (argc < 3) {
+ fprintf(stderr,
+ ANSI_RED "%% Usage: "
+ "%s "
+ "..\n" ANSI_RESET,
+ argv[0]);
+ return 1;
+ }
+
+ brokers = argv[1];
+ groupid = argv[2];
+ topics = &argv[3];
+ topic_cnt = argc - 3;
+
+
+ /*
+ * Create Kafka client configuration place-holder
+ */
+ conf = rd_kafka_conf_new();
+
+ /* Set bootstrap broker(s) as a comma-separated list of
+ * host or host:port (default port 9092).
+ * librdkafka will use the bootstrap brokers to acquire the full
+ * set of brokers from the cluster. */
+ if (rd_kafka_conf_set(conf, "bootstrap.servers", brokers, errstr,
+ sizeof(errstr)) != RD_KAFKA_CONF_OK) {
+ fprintf(stderr, ANSI_RED "%s\n" ANSI_RESET, errstr);
+ rd_kafka_conf_destroy(conf);
+ return 1;
+ }
+
+ if (rd_kafka_conf_set(conf, "debug",
+ "protocol",
+ errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) {
+ fprintf(stderr, ANSI_RED "%s\n" ANSI_RESET, errstr);
+ rd_kafka_conf_destroy(conf);
+ return 1;
+ }
+
+ /* Set the consumer group id.
+ * All consumers sharing the same group id will join the same
+ * group, and the subscribed topic' partitions will be assigned
+ * according to the partition.assignment.strategy
+ * (consumer config property) to the consumers in the group. */
+ if (rd_kafka_conf_set(conf, "group.id", groupid, errstr,
+ sizeof(errstr)) != RD_KAFKA_CONF_OK) {
+ fprintf(stderr, ANSI_RED "%s\n" ANSI_RESET, errstr);
+ rd_kafka_conf_destroy(conf);
+ return 1;
+ }
+
+ /*
+ * Create a new share consumer instance.
+ *
+ * NOTE: rd_kafka_share_consumer_new() takes ownership of the conf
+ * object and the application must not reference it again after this
+ * call.
+ */
+ rkshare = rd_kafka_share_consumer_new(conf, errstr, sizeof(errstr));
+ if (!rkshare) {
+ fprintf(stderr,
+ ANSI_RED "%% Failed to create new share consumer: "
+ "%s\n" ANSI_RESET,
+ errstr);
+ return 1;
+ }
+
+ conf = NULL; /* Configuration object is now owned, and freed,
+ * by the rd_kafka_t instance. */
+
+
+ /* Convert the list of topics to a format suitable for librdkafka */
+ subscription = rd_kafka_topic_partition_list_new(topic_cnt);
+ for (i = 0; i < topic_cnt; i++)
+ rd_kafka_topic_partition_list_add(subscription, topics[i],
+ /* the partition is ignored
+ * by subscribe() */
+ RD_KAFKA_PARTITION_UA);
+
+ /* Subscribe to the list of topics */
+ err = rd_kafka_share_subscribe(rkshare, subscription);
+ if (err) {
+ fprintf(stderr,
+ ANSI_RED "%% Failed to subscribe to %d topics: "
+ "%s\n" ANSI_RESET,
+ subscription->cnt, rd_kafka_err2str(err));
+ rd_kafka_topic_partition_list_destroy(subscription);
+ rd_kafka_share_destroy(rkshare);
+ return 1;
+ }
+
+ fprintf(stderr,
+ ANSI_YELLOW "%% Subscribed to %d topic(s), "
+ "waiting for rebalance and messages...\n" ANSI_RESET,
+ subscription->cnt);
+
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+
+ /* Signal handler for clean shutdown */
+ signal(SIGINT, stop);
+
+ /* Subscribing to topics will trigger a group rebalance
+ * which may take some time to finish, but there is no need
+ * for the application to handle this idle period in a special way
+ * since a rebalance may happen at any time.
+ * Start polling for messages. */
+
+ rd_kafka_message_t *rkmessages[10001];
+ while (run) {
+ rd_kafka_message_t *rkm = NULL;
+ size_t rcvd_msgs = 0;
+ int i;
+ rd_kafka_error_t *error;
+ double __elapsed_ms;
+
+ TIME_BLOCK_MS(__elapsed_ms,
+ error = rd_kafka_share_consume_batch(
+ rkshare, 3000, rkmessages, &rcvd_msgs));
+ fprintf(stdout,
+ ANSI_GREEN "%% rd_kafka_share_consume_batch() took "
+ "%.3f ms\n" ANSI_RESET,
+ __elapsed_ms);
+
+ if (error) {
+ fprintf(stdout,
+ ANSI_RED "%% Consume error: %s\n" ANSI_RESET,
+ rd_kafka_error_string(error));
+ rd_kafka_error_destroy(error);
+ continue;
+ }
+
+ fprintf(stdout,
+ ANSI_GREEN "%% Received %zu messages\n" ANSI_RESET,
+ rcvd_msgs);
+ for (i = 0; i < (int)rcvd_msgs; i++) {
+ rkm = rkmessages[i];
+
+ if (rkm->err) {
+ fprintf(stdout,
+ ANSI_RED "%% Consumer error: %d: "
+ "%s\n" ANSI_RESET,
+ rkm->err, rd_kafka_message_errstr(rkm));
+ rd_kafka_message_destroy(rkm);
+ continue;
+ }
+
+ /* Proper message. */
+ printf(ANSI_CYAN "Message received on %s [%" PRId32
+ "] at offset %" PRId64 ANSI_RESET,
+ 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(ANSI_CYAN " Key: %.*s\n" ANSI_RESET,
+ (int)rkm->key_len,
+ (const char *)rkm->key);
+ else if (rkm->key)
+ printf(ANSI_CYAN " Key: (%d bytes)\n" ANSI_RESET,
+ (int)rkm->key_len);
+
+ /* Print the message value/payload. */
+ if (rkm->payload &&
+ is_printable(rkm->payload, rkm->len))
+ printf(ANSI_CYAN " - Value: %.*s\n" ANSI_RESET,
+ (int)rkm->len,
+ (const char *)rkm->payload);
+ else if (rkm->payload)
+ printf(ANSI_CYAN " - Value: (%d bytes)\n" ANSI_RESET,
+ (int)rkm->len);
+
+ rd_kafka_message_destroy(rkm);
+ }
+ }
+
+
+ /* Close the consumer: commit final offsets and leave the group. */
+ fprintf(stderr, ANSI_YELLOW "%% Closing share consumer\n" ANSI_RESET);
+ rd_kafka_share_consumer_close(rkshare);
+
+
+ /* Destroy the consumer */
+ rd_kafka_share_destroy(rkshare);
+
+ return 0;
+}
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index bbe63cff48..8f123826d8 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -26,6 +26,7 @@ set(
rdkafka_offset.c
rdkafka_op.c
rdkafka_partition.c
+ rdkafka_share_acknowledgement.c
rdkafka_pattern.c
rdkafka_queue.c
rdkafka_range_assignor.c
@@ -51,6 +52,7 @@ set(
rdkafka_mock.c
rdkafka_mock_handlers.c
rdkafka_mock_cgrp.c
+ rdkafka_mock_sharegrp.c
rdkafka_error.c
rdkafka_fetcher.c
rdkafka_telemetry.c
diff --git a/src/Makefile b/src/Makefile
index 0d0635ce30..4de1b48359 100644
--- a/src/Makefile
+++ b/src/Makefile
@@ -41,7 +41,8 @@ SRCS= rdkafka.c rdkafka_broker.c rdkafka_msg.c rdkafka_topic.c \
rdkafka_conf.c rdkafka_timer.c rdkafka_offset.c \
rdkafka_transport.c rdkafka_buf.c rdkafka_queue.c rdkafka_op.c \
rdkafka_request.c rdkafka_cgrp.c rdkafka_pattern.c \
- rdkafka_partition.c rdkafka_subscription.c \
+ rdkafka_partition.c rdkafka_share_acknowledgement.c \
+ rdkafka_subscription.c \
rdkafka_assignment.c \
rdkafka_assignor.c rdkafka_range_assignor.c \
rdkafka_roundrobin_assignor.c rdkafka_sticky_assignor.c \
@@ -56,8 +57,8 @@ SRCS= rdkafka.c rdkafka_broker.c rdkafka_msg.c rdkafka_topic.c \
rdkafka_header.c rdkafka_admin.c rdkafka_aux.c \
rdkafka_background.c rdkafka_idempotence.c rdkafka_cert.c \
rdkafka_txnmgr.c rdkafka_coord.c rdbase64.c \
- rdvarint.c rdbuf.c rdmap.c rdunittest.c \
- rdkafka_mock.c rdkafka_mock_handlers.c rdkafka_mock_cgrp.c \
+ rdvarint.c rdbuf.c rdmap.c rdunittest.c rdunittest_fetcher.c \
+ rdkafka_mock.c rdkafka_mock_handlers.c rdkafka_mock_cgrp.c rdkafka_mock_sharegrp.c \
rdkafka_error.c rdkafka_fetcher.c rdkafka_telemetry.c \
rdkafka_telemetry_encode.c rdkafka_telemetry_decode.c \
nanopb/pb_encode.c nanopb/pb_decode.c nanopb/pb_common.c \
diff --git a/src/rdhdrhistogram.c b/src/rdhdrhistogram.c
index 08240ac7a3..13016e2065 100644
--- a/src/rdhdrhistogram.c
+++ b/src/rdhdrhistogram.c
@@ -275,8 +275,7 @@ typedef struct rd_hdr_iter_s {
int64_t highestEquivalentValue;
} rd_hdr_iter_t;
-#define RD_HDR_ITER_INIT(hdr) \
- { .hdr = hdr, .subBucketIdx = -1 }
+#define RD_HDR_ITER_INIT(hdr) {.hdr = hdr, .subBucketIdx = -1}
static int rd_hdr_iter_next(rd_hdr_iter_t *it) {
const rd_hdr_histogram_t *hdr = it->hdr;
diff --git a/src/rdkafka.c b/src/rdkafka.c
index ee0adcd59f..d67d4e3806 100644
--- a/src/rdkafka.c
+++ b/src/rdkafka.c
@@ -1210,6 +1210,18 @@ void rd_kafka_destroy(rd_kafka_t *rk) {
rd_kafka_destroy_app(rk, 0);
}
+/**
+ * TODO KIP-932: Destroy inflight map entries should be done in consumer close.
+ */
+void rd_kafka_share_destroy(rd_kafka_share_t *rkshare) {
+ /**
+ * TODO KIP-932: Guard this with checks for rkshare and
+ * rkshare->rkshare_rk?
+ */
+ rd_kafka_destroy(rkshare->rkshare_rk);
+ rd_free(rkshare);
+}
+
void rd_kafka_destroy_flags(rd_kafka_t *rk, int flags) {
rd_kafka_destroy_app(rk, flags);
}
@@ -2235,6 +2247,13 @@ static int rd_kafka_init_wait(rd_kafka_t *rk, int timeout_ms) {
}
+/* Forward declarations: defined after reply/fanout handlers */
+static rd_kafka_broker_t *rd_kafka_share_select_broker(rd_kafka_t *rk,
+ rd_kafka_cgrp_t *rkcg);
+static void rd_kafka_share_enqueue_fetch_op(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb,
+ rd_bool_t should_fetch);
+
/**
* Main loop for Kafka handler thread.
*/
@@ -2291,6 +2310,35 @@ static int rd_kafka_thread_main(void *arg) {
RD_KAFKA_Q_CB_CALLBACK, NULL, NULL);
if (rk->rk_cgrp) /* FIXME: move to timer-triggered */
rd_kafka_cgrp_serve(rk->rk_cgrp);
+
+ /* KIP-932: If fetch_more_records is set but no fetch
+ * op is in-flight and assignments exist, re-trigger
+ * the fetch flow by selecting a broker directly.
+ * TODO: KIP-932: Check if rkcg_current_assignment is
+ * correct here or if
+ * rkcg->rkcg_rk->rk_consumer.assignment.all should
+ * be used instead. */
+ if (RD_KAFKA_IS_SHARE_CONSUMER(rk) && rk->rk_cgrp &&
+ rk->rk_cgrp->rkcg_share.fetch_more_records &&
+ rk->rk_cgrp->rkcg_share.share_fetch_ops_in_flight ==
+ 0 &&
+ rk->rk_cgrp->rkcg_current_assignment &&
+ rk->rk_cgrp->rkcg_current_assignment->cnt > 0) {
+ rd_kafka_broker_t *rkb_sel;
+
+ rkb_sel = rd_kafka_share_select_broker(
+ rk, rk->rk_cgrp);
+ if (rkb_sel) {
+ rd_kafka_dbg(rk, CONSUMER, "FETCHMORE",
+ "Re-triggering fetch for "
+ "fetch_more_records=true "
+ "and no fetch in-flight");
+ rd_kafka_share_enqueue_fetch_op(rk, rkb_sel,
+ rd_true);
+ rd_kafka_broker_destroy(rkb_sel);
+ }
+ }
+
rd_kafka_timers_run(&rk->rk_timers, RD_POLL_NOWAIT);
}
@@ -2868,6 +2916,712 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
return NULL;
}
+rd_kafka_share_t *rd_kafka_share_consumer_new(rd_kafka_conf_t *conf,
+ char *errstr,
+ size_t errstr_size) {
+ rd_kafka_share_t *rkshare;
+ 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;
+ }
+
+ /**
+ * TODO KIP-932: Check if this way of defining share consumer needs to
+ * be changed.
+ */
+ 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;
+ }
+
+ /**
+ * TODO KIP-932: Remove this property once we have removed offset
+ * management.
+ */
+ if (rd_kafka_conf_set(conf, "auto.offset.reset", "earliest", errstr,
+ sizeof(errstr)) != RD_KAFKA_CONF_OK) {
+ fprintf(stderr, "%s\n", errstr);
+ rd_kafka_conf_destroy(conf);
+ return NULL;
+ }
+
+ /**
+ * TODO KIP-932: Remove this property once we have removed offset
+ * management.
+ */
+ 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 NULL;
+ }
+
+ /**
+ * TODO KIP-932: Try removing use of this property when improving share
+ * consumer rebalancing logic in group management ticket.
+ */
+ 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;
+ }
+
+ rkshare = rd_calloc(1, sizeof(*rkshare));
+ rkshare->rkshare_rk = rk;
+ rkshare->rkshare_unacked_cnt = 0;
+
+ /* Set backpointer from rk to rkshare for access in retry handlers */
+ rk->rk_rkshare = rkshare;
+
+ /* Inflight acks map keyed by topic_id + partition */
+ RD_MAP_INIT(&rkshare->rkshare_inflight_acks, 16,
+ rd_kafka_topic_partition_by_id_cmp,
+ rd_kafka_topic_partition_hash_by_id,
+ rd_kafka_topic_partition_destroy_free,
+ rd_kafka_share_ack_batches_destroy_free);
+
+ return rkshare;
+}
+
+
+/**
+ * @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_dbg(
+ rk, CGRP, "SHARE",
+ "Selected broker %s (%p) for share fetch "
+ "for partition %s [%" PRId32 "]",
+ rd_kafka_broker_name(selected_rkb),
+ selected_rkb,
+ partition->topic, partition->partition);
+ }
+ 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
+ */
+/**
+ * @brief Enqueue a SHARE_FETCH_FANOUT op and set the fetch guard flag
+ * if the op requests more records.
+ */
+static void rd_kafka_share_fetch_fanout_enqueue(rd_kafka_t *rk,
+ rd_kafka_op_t *rko) {
+ if (rko->rko_u.share_fetch_fanout.fetch_more_records)
+ rk->rk_rkshare->rkshare_fetch_more_records_requested =
+ rd_true;
+ rd_kafka_q_enq(rk->rk_ops, rko);
+}
+
+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_share_fetch_fanout_enqueue(rk, 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_bool_t fetch_more_records,
+ int backoff_ms,
+ rd_list_t *ack_batches) {
+ 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.fetch_more_records = fetch_more_records;
+ rko->rko_replyq = RD_KAFKA_REPLYQ(rk->rk_ops, 0);
+
+ /* Attach pre-built ack_details (ownership transferred to op) */
+ rko->rko_u.share_fetch_fanout.ack_batches = ack_batches;
+
+ 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_share_fetch_fanout_enqueue(rk, 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_broker_t *reply_rkb =
+ rko_orig->rko_u.share_fetch.target_broker;
+ rd_kafka_cgrp_t *rkcg = rd_kafka_cgrp_get(rk);
+ rd_bool_t should_fetch = rko_orig->rko_u.share_fetch.should_fetch;
+ rd_bool_t records_fetched =
+ rko_orig->rko_u.share_fetch.records_fetched;
+
+ rd_kafka_assert(rk, thrd_is_current(rk->rk_thread));
+ rd_kafka_dbg(rk, CGRP, "SHAREFETCH",
+ "Share fetch reply: %s, should_fetch=%d, "
+ "records_fetched=%d, broker=%s",
+ rd_kafka_err2str(rko_orig->rko_err), should_fetch,
+ records_fetched,
+ reply_rkb ? rd_kafka_broker_name(reply_rkb) : "none");
+
+ reply_rkb->rkb_share_fetch_enqueued = rd_false;
+
+ if (should_fetch)
+ rkcg->rkcg_share.share_fetch_ops_in_flight--;
+
+ /*
+ * Step 1: If records were fetched, reset the global fetch guard
+ * so the next FANOUT can select a new fetch broker.
+ */
+ if (records_fetched)
+ rkcg->rkcg_share.fetch_more_records = rd_false;
+
+ /*
+ * Step 2: If this was the fetch broker but no records were received,
+ * try to select another broker to fetch from.
+ *
+ * TODO: KIP-932: Handle the case where all assignments are removed
+ * while a fetch is in progress. In that scenario, no records are
+ * returned and rd_kafka_share_select_broker() will always return
+ * NULL (no assigned partitions), leaving fetch_more_records stuck
+ * at true indefinitely. Need to detect this (e.g. check if
+ * assignments are empty) and reset fetch_more_records.
+ */
+ if (should_fetch && !records_fetched &&
+ rkcg->rkcg_share.fetch_more_records) {
+ rd_kafka_broker_t *selected_rkb;
+
+ selected_rkb = rd_kafka_share_select_broker(rk, rkcg);
+ if (selected_rkb) {
+ rd_kafka_dbg(
+ rk, CGRP, "SHARE",
+ "Re-selecting broker %s for share fetch "
+ "(previous broker %s returned no records)",
+ rd_kafka_broker_name(selected_rkb),
+ rd_kafka_broker_name(reply_rkb));
+
+ rd_kafka_share_enqueue_fetch_op(rk, selected_rkb,
+ rd_true);
+ rd_kafka_broker_destroy(selected_rkb);
+ } else {
+ rd_kafka_dbg(rk, CGRP, "SHARE",
+ "No broker available for share fetch "
+ "retry, will retry on next FANOUT");
+ }
+ }
+
+ /*
+ * Step 3: If the replying broker has cached ack details,
+ * send an ack-only SHARE_FETCH op to it.
+ */
+ if (reply_rkb->rkb_share_async_ack_details &&
+ !reply_rkb->rkb_share_fetch_enqueued) {
+ rd_kafka_dbg(rk, CGRP, "SHARE",
+ "Enqueuing ack-only SHARE_FETCH to broker %s "
+ "to flush pending acks",
+ rd_kafka_broker_name(reply_rkb));
+ rd_kafka_share_enqueue_fetch_op(rk, reply_rkb, rd_false);
+ }
+
+ 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",
+ rd_kafka_err2name(err));
+
+ 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.fetch_more_records,
+ /* TODO: KIP-932: Consider setting this to retry_backoff_ms
+ or to a constant.*/
+ rk->rk_conf.retry_backoff_max_ms,
+ NULL /* no ack_details on retry */);
+ break;
+ }
+ return RD_KAFKA_OP_RES_HANDLED;
+}
+
+/**
+ * @brief Find an existing ack batch for the same topic-partition in a list.
+ *
+ * @param ack_list List of rd_kafka_share_ack_batches_t*
+ * @param rktpar Topic-partition to match (by topic id and partition)
+ *
+ * @returns Matching batch, or NULL if not found.
+ * @locality main thread
+ */
+static rd_kafka_share_ack_batches_t *
+rd_kafka_share_find_ack_batch(rd_list_t *ack_list,
+ const rd_kafka_topic_partition_t *rktpar) {
+ rd_kafka_share_ack_batches_t *existing;
+ int i;
+
+ RD_LIST_FOREACH(existing, ack_list, i) {
+ if (rd_kafka_topic_partition_by_id_cmp(existing->rktpar,
+ rktpar) == 0)
+ return existing;
+ }
+ return NULL;
+}
+
+/**
+ * @brief Segregate ack batches from a FANOUT op by partition leader.
+ *
+ * For each ack batch, looks up the current leader broker via the
+ * toppar reference in batch->rktpar, and merges the batch into that
+ * broker's rkb_share_async_ack_details list. If the broker already
+ * has cached acks for the same topic-partition, the entries are
+ * appended to the existing batch.
+ *
+ * @param rk Client instance
+ * @param ack_batches List of rd_kafka_share_ack_batches_t* from FANOUT op.
+ * Elements whose leader is found are moved to broker
+ * ack_details; remaining elements are not freed.
+ *
+ * @locality main thread
+ */
+static void
+rd_kafka_share_segregate_acks_by_leader(rd_kafka_t *rk,
+ rd_list_t *ack_batches) {
+ rd_kafka_share_ack_batches_t *batch;
+ int batch_cnt = rd_list_cnt(ack_batches);
+
+ while ((batch = rd_list_pop(ack_batches))) {
+ rd_kafka_toppar_t *rktp;
+ rd_kafka_broker_t *leader_rkb;
+ rd_kafka_share_ack_batches_t *existing;
+
+ rktp = rd_kafka_topic_partition_toppar(rk, batch->rktpar);
+ if (!rktp || !rktp->rktp_leader) {
+ rd_kafka_dbg(rk, CGRP, "SHARE",
+ "Ack batch for leader %" PRId32
+ " dropped: toppar or leader not available",
+ batch->response_leader_id);
+ rd_kafka_share_ack_batches_destroy(batch, rd_true);
+ continue;
+ }
+ leader_rkb = rktp->rktp_leader;
+
+ /* Allocate list on first use with incoming batch count
+ * as initial capacity hint */
+ if (!leader_rkb->rkb_share_async_ack_details)
+ leader_rkb->rkb_share_async_ack_details =
+ rd_list_new(batch_cnt, NULL);
+
+ /* Check if there's already a batch for this topic-partition.
+ * If so, merge entries; otherwise add as new. */
+ existing = rd_kafka_share_find_ack_batch(
+ leader_rkb->rkb_share_async_ack_details, batch->rktpar);
+
+ if (existing) {
+ /* Merge: move entries from new batch into existing */
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int j;
+ RD_LIST_FOREACH(entry, &batch->entries, j) {
+ rd_list_add(&existing->entries, entry);
+ }
+ /* Entries have been moved to existing by pointer.
+ * Destroy batch shell, list container, and rktpar
+ * (each batch owns its own copy). */
+ rd_list_destroy(&batch->entries);
+ if (batch->rktpar)
+ rd_kafka_topic_partition_destroy(
+ batch->rktpar);
+ rd_free(batch);
+ } else {
+ rd_list_add(leader_rkb->rkb_share_async_ack_details,
+ batch);
+ }
+ }
+}
+
+
+/**
+ * @brief Create and enqueue a SHARE_FETCH op on a broker.
+ *
+ * Moves any cached ack details from rkb_share_async_ack_details
+ * into the op, sets rkb_share_fetch_enqueued, and enqueues the op
+ * on the broker's ops queue.
+ *
+ * @param rk Client instance
+ * @param rkb Broker to enqueue on. Must not have rkb_share_fetch_enqueued set.
+ * @param should_fetch Whether the broker should fetch records.
+ *
+ * @locality main thread
+ */
+static void rd_kafka_share_enqueue_fetch_op(rd_kafka_t *rk,
+ rd_kafka_broker_t *rkb,
+ rd_bool_t should_fetch) {
+ 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_false;
+ rko_sf->rko_u.share_fetch.abs_timeout =
+ rd_timeout_init(rk->rk_conf.socket_timeout_ms);
+ rko_sf->rko_u.share_fetch.should_fetch = should_fetch;
+
+ /* Move ack details from broker cache to op */
+ rko_sf->rko_u.share_fetch.ack_details =
+ rkb->rkb_share_async_ack_details;
+ rkb->rkb_share_async_ack_details = NULL;
+
+ 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);
+
+ rkb->rkb_share_fetch_enqueued = rd_true;
+
+ if (should_fetch)
+ rd_kafka_cgrp_get(rk)->rkcg_share
+ .share_fetch_ops_in_flight++;
+
+ rd_kafka_dbg(rk, CGRP, "SHAREFETCH",
+ "Enqueuing share fetch op on broker %s "
+ "(%s fetch, %s acks)",
+ rd_kafka_broker_name(rkb),
+ should_fetch ? "with" : "no",
+ rko_sf->rko_u.share_fetch.ack_details ? "with" : "no");
+
+ /* TODO KIP-932: Remove this debug printing */
+ {
+ static const char *ack_type_names[] = {
+ [0] = "GAP",
+ [1] = "ACCEPT",
+ [2] = "RELEASE",
+ [3] = "REJECT",
+ };
+ rd_list_t *ack_details =
+ rko_sf->rko_u.share_fetch.ack_details;
+
+ printf("[SHARE_FETCH] broker=%s should_fetch=%d",
+ rd_kafka_broker_name(rkb), should_fetch);
+
+ if (ack_details && rd_list_cnt(ack_details) > 0) {
+ rd_kafka_share_ack_batches_t *batches;
+ int k;
+ printf(" ack_details=[");
+ RD_LIST_FOREACH(batches, ack_details, k) {
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int m;
+ if (k > 0)
+ printf(", ");
+ printf("%s[%" PRId32 "]:{",
+ batches->rktpar->topic,
+ batches->rktpar->partition);
+ RD_LIST_FOREACH(entry, &batches->entries, m) {
+ int type_val =
+ (int)entry->types[0];
+ const char *type_str =
+ (type_val >= 0 && type_val <= 3)
+ ? ack_type_names[type_val]
+ : "UNKNOWN";
+ if (m > 0)
+ printf(", ");
+ printf("%" PRId64 "-%" PRId64
+ "(%s)",
+ entry->start_offset,
+ entry->end_offset,
+ type_str);
+ }
+ printf("}");
+ }
+ printf("]");
+ } else {
+ printf(" ack_details=none");
+ }
+ printf("\n");
+ fflush(stdout);
+ }
+
+ rd_kafka_q_enq(rkb->rkb_ops, rko_sf);
+}
+
+/**
+ * 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_cgrp_t *rkcg = rd_kafka_cgrp_get(rk);
+ rd_bool_t fetch_more_records =
+ rko->rko_u.share_fetch_fanout.fetch_more_records;
+ rd_bool_t has_fanout_acks =
+ (rko->rko_u.share_fetch_fanout.ack_batches &&
+ rd_list_cnt(rko->rko_u.share_fetch_fanout.ack_batches) > 0);
+
+ /*
+ * Step 1: Segregate acks by partition leader.
+ * Each ack batch has a rktpar with an rktp reference; get the
+ * current leader from rktp->rktp_leader and merge the batch
+ * into that broker's rkb_share_async_ack_details.
+ */
+ if (has_fanout_acks) {
+ rd_kafka_share_segregate_acks_by_leader(
+ rk, rko->rko_u.share_fetch_fanout.ack_batches);
+
+ /* Ownership of elements transferred to broker ack_details.
+ * Destroy the container (rd_list_new sets F_ALLOCATED,
+ * so rd_list_destroy frees the struct) and set to NULL. */
+ rd_list_destroy(rko->rko_u.share_fetch_fanout.ack_batches);
+ rko->rko_u.share_fetch_fanout.ack_batches = NULL;
+ }
+
+ /*
+ * Step 2: Global fetch guard.
+ * Select a fetch broker if:
+ * (a) fetch requested and no fetch is already in progress, OR
+ * (b) fetch_more_records is already set but no broker actually
+ * has a fetch enqueued (stuck state from failed re-selection).
+ */
+ if (fetch_more_records) {
+ rd_bool_t need_select = rd_false;
+
+ if (!rkcg->rkcg_share.fetch_more_records) {
+ rkcg->rkcg_share.fetch_more_records = rd_true;
+ need_select = rd_true;
+ } else if (rkcg->rkcg_share.share_fetch_ops_in_flight ==
+ 0) {
+ need_select = rd_true;
+ }
+
+ if (need_select) {
+ selected_rkb =
+ rd_kafka_share_select_broker(rk, rkcg);
+
+ if (selected_rkb)
+ rd_kafka_dbg(
+ rk, CGRP, "SHARE",
+ "Selected broker %s for fetching "
+ "messages",
+ rd_kafka_broker_name(selected_rkb));
+ // else
+ // rd_kafka_dbg(
+ // rk, CGRP, "SHARE",
+ // "No broker available for share fetch");
+ }
+ }
+
+ if (!fetch_more_records && !has_fanout_acks) {
+ rd_kafka_dbg(rk, CGRP, "SHARE",
+ "No fetch or acks to fan out");
+ return RD_KAFKA_OP_RES_HANDLED;
+ }
+
+ /*
+ * Step 3: Send SHARE_FETCH ops to brokers that have pending acks
+ * or are the selected fetch broker. Cache acks for brokers with
+ * in-flight requests.
+ */
+ rd_kafka_rdlock(rk);
+ TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) {
+ rd_bool_t has_acks =
+ (rkb->rkb_share_async_ack_details != NULL);
+ rd_bool_t is_fetch_broker = (rkb == selected_rkb);
+
+ if (rd_kafka_broker_or_instance_terminating(rkb) ||
+ RD_KAFKA_BROKER_IS_LOGICAL(rkb))
+ continue;
+
+ /* Skip brokers with nothing to do */
+ if (!has_acks && !is_fetch_broker)
+ continue;
+
+ /* If a request is already in-flight on this broker,
+ * acks stay cached in rkb_share_async_ack_details
+ * and will be sent with the next request. */
+ if (rkb->rkb_share_fetch_enqueued) {
+ rd_kafka_dbg(
+ rk, CGRP, "SHARE",
+ "Broker %s has in-flight request, "
+ "with acks: %s, "
+ "is fetch broker: %s",
+ rd_kafka_broker_name(rkb),
+ has_acks ? "yes" : "no",
+ is_fetch_broker ? "yes" : "no");
+ continue;
+ }
+
+ rd_kafka_share_enqueue_fetch_op(rk, rkb, is_fetch_broker);
+ }
+ 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_share_t *rkshare,
+ 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_t *rk = rkshare->rkshare_rk;
+ rd_kafka_cgrp_t *rkcg;
+ size_t max_poll_records = (size_t)rk->rk_conf.share.max_poll_records;
+ rd_bool_t has_records;
+ rd_bool_t has_pending_acks;
+ rd_kafka_error_t *error;
+
+ 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");
+
+ /* Drain rk_rep for all pending callbacks (non-blocking) */
+ rd_kafka_q_serve(rk->rk_rep, RD_POLL_NOWAIT, 0,
+ RD_KAFKA_Q_CB_CALLBACK, rd_kafka_poll_cb, NULL);
+
+ /* Implicit ack: acknowledge all ACQUIRED records from previous
+ * poll as ACCEPT, then extract ack_details for sending. */
+ rd_kafka_share_ack_all(rk->rk_rkshare);
+ rd_list_t *ack_batches = rd_kafka_share_build_ack_details(rk->rk_rkshare);
+
+ has_records = rd_kafka_q_len(rkcg->rkcg_q) > 0;
+ has_pending_acks = ack_batches != NULL;
+
+ /* Only request fetch if no fetch FANOUT is already in flight */
+ rd_bool_t need_fetch_more_records =
+ !has_records &&
+ !rk->rk_rkshare->rkshare_fetch_more_records_requested;
+
+ if (need_fetch_more_records || has_pending_acks)
+ rd_kafka_share_fetch_fanout_with_backoff(
+ rk, need_fetch_more_records, 0, ack_batches);
+
+ error = rd_kafka_q_serve_share_rkmessages(
+ rkcg->rkcg_q, timeout_ms, rkmessages, max_poll_records,
+ rkmessages_size);
+
+ /* Drain rk_rep for callbacks again before returning */
+ rd_kafka_q_serve(rk->rk_rep, RD_POLL_NOWAIT, 0,
+ RD_KAFKA_Q_CB_CALLBACK, rd_kafka_poll_cb, NULL);
+
+ return error;
+}
+
/**
* Schedules a rebootstrap of the cluster immediately.
*
@@ -3505,6 +4259,11 @@ rd_kafka_resp_err_t rd_kafka_poll_set_consumer(rd_kafka_t *rk) {
}
+rd_kafka_resp_err_t
+rd_kafka_share_poll_set_consumer(rd_kafka_share_t *rkshare) {
+ return rd_kafka_poll_set_consumer(rkshare->rkshare_rk);
+}
+
rd_kafka_message_t *rd_kafka_consumer_poll(rd_kafka_t *rk, int timeout_ms) {
rd_kafka_cgrp_t *rkcg;
@@ -3638,6 +4397,10 @@ rd_kafka_resp_err_t rd_kafka_consumer_close(rd_kafka_t *rk) {
return err;
}
+rd_kafka_resp_err_t rd_kafka_share_consumer_close(rd_kafka_share_t *rkshare) {
+ return rd_kafka_consumer_close(rkshare->rkshare_rk);
+}
+
int rd_kafka_consumer_closed(rd_kafka_t *rk) {
if (unlikely(!rk->rk_cgrp))
@@ -4099,6 +4862,9 @@ rd_kafka_op_res_t rd_kafka_poll_cb(rd_kafka_t *rk,
return RD_KAFKA_OP_RES_PASS; /* Return as event */
}
+ // rd_kafka_dbg(rk, CGRP, "CB", "Received cb_type %d op_type 0x%04x", cb_type,
+ // rko->rko_type);
+
switch ((int)rko->rko_type) {
case RD_KAFKA_OP_FETCH:
if (!rk->rk_conf.consume_cb ||
@@ -4317,6 +5083,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. */
@@ -5497,13 +6272,23 @@ const char *rd_kafka_Uuid_base64str(const rd_kafka_Uuid_t *uuid) {
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 = '_';
+ }
+
rd_strlcpy((char *)uuid->base64str, out_base64_str,
23 /* Removing extra ('=') padding */);
rd_free(out_base64_str);
+
return uuid->base64str;
}
diff --git a/src/rdkafka.h b/src/rdkafka.h
index c77f4d7925..ae2099ef12 100644
--- a/src/rdkafka.h
+++ b/src/rdkafka.h
@@ -265,6 +265,7 @@ typedef struct rd_kafka_acl_result_s rd_kafka_acl_result_t;
typedef struct rd_kafka_Uuid_s rd_kafka_Uuid_t;
typedef struct rd_kafka_topic_partition_result_s
rd_kafka_topic_partition_result_t;
+typedef struct rd_kafka_share_s rd_kafka_share_t;
/* @endcond */
@@ -3057,6 +3058,27 @@ rd_kafka_t *rd_kafka_new(rd_kafka_type_t type,
size_t errstr_size);
+RD_EXPORT
+rd_kafka_share_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_share_t *rkshare,
+ int timeout_ms,
+ rd_kafka_message_t **rkmessages /* out */,
+ size_t *rkmessages_size /* out */);
+
/**
* @brief Destroy Kafka handle.
*
@@ -3082,6 +3104,13 @@ void rd_kafka_destroy(rd_kafka_t *rk);
RD_EXPORT
void rd_kafka_destroy_flags(rd_kafka_t *rk, int flags);
+
+/**
+ * TODO KIP-932: Add proper documentation.
+ */
+RD_EXPORT
+void rd_kafka_share_destroy(rd_kafka_share_t *rkshare);
+
/**
* @brief Flags for rd_kafka_destroy_flags()
*/
@@ -4712,6 +4741,100 @@ RD_EXPORT rd_kafka_error_t *rd_kafka_consumer_group_metadata_read(
+/**
+ * @name ShareConsumer (C)
+ * @brief Share consumer specific APIs
+ * @{
+ *
+ *
+ *
+ */
+/**
+ * TODO KIP-932:
+ * 1) Update descriptions of all the below APIs.
+ * 2) Shall we guard these APIs with asserts for rkshare type and
+ * internal rkshare_rk to be present.
+ */
+/**
+ * @brief Subscribe to topic set using balanced consumer groups.
+ *
+ * Wildcard (regex) topics are not supported.
+ *
+ * @remark Only the \c .topic field is used in the supplied \p topics list,
+ * all other fields are ignored.
+ *
+ * @remark subscribe() is an asynchronous method which returns immediately:
+ * background threads will (re)join the group, wait for group rebalance,
+ * issue any registered rebalance_cb, assign() the assigned partitions,
+ * and then start fetching messages.
+ *
+ * @remark After this call returns a consumer error will be returned by
+ * rd_kafka_consumer_poll (et.al) for each unavailable topic in the
+ * \p topics. The error will be RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART
+ * for non-existent topics, and
+ * RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED for unauthorized topics.
+ * The consumer error will be raised through rd_kafka_consumer_poll()
+ * (et.al.) with the \c rd_kafka_message_t.err field set to one of the
+ * error codes mentioned above.
+ * The subscribe function itself is asynchronous and will not return
+ * an error on unavailable topics.
+ *
+ * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or
+ * RD_KAFKA_RESP_ERR__INVALID_ARG if list is empty, contains invalid
+ * topics or regexes or duplicate entries,
+ * RD_KAFKA_RESP_ERR__FATAL if the consumer has raised a fatal error.
+ */
+RD_EXPORT rd_kafka_resp_err_t
+rd_kafka_share_subscribe(rd_kafka_share_t *rkshare,
+ const rd_kafka_topic_partition_list_t *topics);
+
+
+/**
+ * @brief Unsubscribe from the current subscription set.
+ */
+RD_EXPORT
+rd_kafka_resp_err_t rd_kafka_share_unsubscribe(rd_kafka_share_t *rkshare);
+
+
+/**
+ * @brief Returns the current topic subscription
+ *
+ * @returns An error code on failure, otherwise \p topic is updated
+ * to point to a newly allocated topic list (possibly empty).
+ *
+ * @remark The application is responsible for calling
+ * rd_kafka_topic_partition_list_destroy on the returned list.
+ */
+RD_EXPORT rd_kafka_resp_err_t
+rd_kafka_share_subscription(rd_kafka_share_t *rkshare,
+ rd_kafka_topic_partition_list_t **topics);
+
+
+
+/**
+ * @brief Close the consumer.
+ *
+ * This call will block until the consumer has revoked its assignment,
+ * calling the \c rebalance_cb if it is configured, committed offsets
+ * to broker, and left the consumer group (if applicable).
+ * The maximum blocking time is roughly limited to session.timeout.ms.
+ *
+ * @returns An error code indicating if the consumer close was succesful
+ * or not.
+ * RD_KAFKA_RESP_ERR__FATAL is returned if the consumer has raised
+ * a fatal error.
+ *
+ * @remark The application still needs to call rd_kafka_destroy() after
+ * this call finishes to clean up the underlying handle resources.
+ *
+ */
+RD_EXPORT
+rd_kafka_resp_err_t rd_kafka_share_consumer_close(rd_kafka_share_t *rkshare);
+
+/**@}*/
+
+
+
/**
* @name Producer API
* @{
@@ -5547,6 +5670,16 @@ RD_EXPORT
rd_kafka_resp_err_t rd_kafka_poll_set_consumer(rd_kafka_t *rk);
+/**
+ * @brief Redirect the main (rd_kafka_poll()) queue to the KafkaConsumer's
+ * queue (rd_kafka_consumer_poll()).
+ *
+ * @warning It is not permitted to call rd_kafka_poll() after directing the
+ * main queue with rd_kafka_share_poll_set_consumer().
+ */
+RD_EXPORT
+rd_kafka_resp_err_t rd_kafka_share_poll_set_consumer(rd_kafka_share_t *rkshare);
+
/**@}*/
/**
diff --git a/src/rdkafka_broker.c b/src/rdkafka_broker.c
index fb577a9fde..805bc0ca2d 100644
--- a/src/rdkafka_broker.c
+++ b/src/rdkafka_broker.c
@@ -3175,6 +3175,95 @@ 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).
@@ -3320,6 +3409,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);
@@ -3418,6 +3510,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) {
@@ -3464,6 +3557,72 @@ 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);
+ } else {
+ 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",
@@ -3550,6 +3709,26 @@ 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");
break;
@@ -4268,8 +4447,79 @@ static void rd_kafka_broker_producer_serve(rd_kafka_broker_t *rkb,
rd_kafka_broker_unlock(rkb);
}
+/**
+ * TODO KIP-932: Remove if not needed later during finalizing share session
+ * implementation.
+ */
+// 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);
+
+ /*
+ * TODO KIP-932: Check the below connection handling properly.
+ */
+ 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++;
+ }
+
+ /**
+ * TODO KIP-932: Check if the below is needed. Currently, used
+ * as it is from the original consumer serve
+ * function.
+ */
+ /* 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
*/
@@ -4472,6 +4722,9 @@ static void rd_kafka_broker_serve(rd_kafka_broker_t *rkb, int timeout_ms) {
rkb->rkb_persistconn.internal =
rd_atomic32_get(&rkb->rkb_outbufs.rkbq_cnt) > 0;
+ /*
+ * TODO KIP-932: Check internal broker handling for shared consumer.
+ */
if (rkb->rkb_source == RD_KAFKA_INTERNAL) {
rd_kafka_broker_internal_serve(rkb, abs_timeout);
return;
@@ -4479,6 +4732,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);
@@ -4710,6 +4965,58 @@ 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);
+
+ /**
+ * TODO KIP-932: Remove the below destroy logic for the
+ * shared fetch session toppar information
+ * when the share_close() is properly
+ * implemented.
+ */
+ 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));
}
}
@@ -4792,6 +5099,11 @@ 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);
+ rd_assert(!rkb->rkb_share_async_ack_details);
if (rkb->rkb_source != RD_KAFKA_INTERNAL &&
(rkb->rkb_rk->rk_conf.security_protocol ==
@@ -4918,11 +5230,18 @@ 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;
+ rkb->rkb_share_async_ack_details = NULL;
+ rkb->rkb_share_fetch_enqueued = rd_false;
CIRCLEQ_INIT(&rkb->rkb_active_toppars);
TAILQ_INIT(&rkb->rkb_monitors);
rd_kafka_bufq_init(&rkb->rkb_outbufs);
@@ -6189,6 +6508,54 @@ 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;
+
+ /* Clear pending ack details cached on this broker.
+ * TODO KIP-932: Maybe move ack_details cleanup to clear
+ * through DESTROY err flow when main thread
+ * receives response from ongoing op. Problem
+ * with that approach is that broker_final
+ * might be called before reply for the op
+ * is processed. Check properly.
+ * TODO KIP-932: Send SHARE_SESSION_NOT_FOUND error in
+ * acknowledgement callback (as done in
+ * Java client). */
+ if (rkb->rkb_share_async_ack_details) {
+ rd_kafka_share_ack_batches_t *batch;
+ int i;
+ RD_LIST_FOREACH(batch,
+ rkb->rkb_share_async_ack_details, i) {
+ rd_kafka_share_ack_batches_destroy(batch,
+ rd_true);
+ }
+ rd_list_destroy(rkb->rkb_share_async_ack_details);
+ rd_free(rkb->rkb_share_async_ack_details);
+ rkb->rkb_share_async_ack_details = NULL;
+ }
+
+ /**
+ * TODO KIP-932: Not leaving properly. Need to fix this.
+ * It will be moved to consumer close
+ * instead of decommissioning.
+ */
+ 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..982cad8364 100644
--- a/src/rdkafka_broker.h
+++ b/src/rdkafka_broker.h
@@ -106,6 +106,70 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */
/* Toppars handled by this broker */
TAILQ_HEAD(, rd_kafka_toppar_s) rkb_toppars;
+
+ /**
+ * TODO KIP-932: Check the type again to optimize the performance
+ * of adding or removing a partition. Maybe use a map
+ * or linked list instead of rd_list_t in some of the
+ * cases.
+ */
+ 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.*/
+ 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 to be added to the
+ * fetch session. `adding_toppars` are
+ * removed from this when fetch request is
+ * successful */
+
+ rd_list_t
+ *adding_toppars; /* List of toppars that are being added to
+ * the session. These are already sent in
+ * the fetch request. Will be removed from
+ * `toppars_to_add` when fetch request is
+ * successful. This is cleared and set to
+ * NULL after the response.
+ */
+
+ 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 the session.
+ * `forgetting_toppars` are removed from
+ * this when fetch request is successful
+ */
+
+ rd_list_t *forgetting_toppars; /* List of toppars that are being
+ * removed from the session.
+ * These are already sent in the
+ * fetch request. Will be removed
+ * from `toppars_to_forget` when
+ * fetch request is successful.
+ * This is cleared and set to
+ * NULL after the response.
+ */
+ 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 +451,23 @@ 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;
+
+ rd_list_t *rkb_share_async_ack_details; /**< Pending ack batches for
+ * this broker (as partition
+ * leader). Type:
+ * rd_kafka_share_ack_batches_t*.
+ * Allocated by main thread
+ * FANOUT handler, moved to
+ * SHARE_FETCH op and set to
+ * NULL. Freed by broker
+ * thread after use.
+ * @locality main thread */
};
#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 0d85cbde32..b664f12ac5 100644
--- a/src/rdkafka_cgrp.c
+++ b/src/rdkafka_cgrp.c
@@ -1080,7 +1080,90 @@ 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) {
@@ -3363,6 +3446,301 @@ 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.
@@ -3673,6 +4051,22 @@ 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);
@@ -3695,6 +4089,24 @@ 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);
@@ -6144,6 +6556,16 @@ 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,
@@ -6205,6 +6627,16 @@ 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 &=
@@ -6294,6 +6726,8 @@ 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..5d8d9e04ab 100644
--- a/src/rdkafka_cgrp.h
+++ b/src/rdkafka_cgrp.h
@@ -377,6 +377,25 @@ 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 */
+ rd_bool_t fetch_more_records; /**< Global fetch-in-flight
+ * guard. When rd_true, a
+ * fetch is already in
+ * progress and new FANOUT
+ * ops should not send
+ * another fetch request. */
+ int share_fetch_ops_in_flight; /**< Number of SHARE_FETCH
+ * ops with should_fetch=true
+ * currently in-flight.
+ * Incremented when enqueuing,
+ * decremented on reply.
+ * @locality main thread */
+ } rkcg_share;
} rd_kafka_cgrp_t;
diff --git a/src/rdkafka_conf.c b/src/rdkafka_conf.c
index 0e346e7456..dd6f7a071b 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..7c205dc57b 100644
--- a/src/rdkafka_fetcher.c
+++ b/src/rdkafka_fetcher.c
@@ -885,6 +885,889 @@ rd_kafka_fetch_reply_handle(rd_kafka_broker_t *rkb,
}
+int64_t rd_kafka_op_get_offset(const rd_kafka_op_t *rko) {
+ if (!rko) {
+ return RD_KAFKA_OFFSET_INVALID;
+ }
+ if (rko->rko_type == RD_KAFKA_OP_FETCH)
+ return rko->rko_u.fetch.rkm.rkm_rkmessage.offset;
+ if (rko->rko_type == RD_KAFKA_OP_CONSUMER_ERR)
+ return rko->rko_u.err.offset;
+ return RD_KAFKA_OFFSET_INVALID;
+}
+
+void rd_kafka_share_filter_msg_from_acq_records(rd_kafka_q_t *temp_fetchq,
+ rd_list_t *filtered_msgs,
+ int32_t AcquiredRecordsArrayCnt,
+ const int64_t *FirstOffsets,
+ const int64_t *LastOffsets) {
+
+ rd_kafka_op_t *rko;
+
+ /* Iterate through all messages in temp_fetchq and forward
+ * only those whose offset falls within an acquired range.
+ * Also set the ack type based on the op type. */
+ while ((rko = rd_kafka_q_pop(temp_fetchq, RD_POLL_NOWAIT, 0)) != NULL) {
+ int64_t rko_offset = rd_kafka_op_get_offset(rko);
+ rd_bool_t in_acquired_range = rd_false;
+ int32_t range_idx;
+
+ /* Check if this message's offset is within any acquired range
+ */
+ for (range_idx = 0; range_idx < AcquiredRecordsArrayCnt;
+ range_idx++) {
+ if (rko_offset >= FirstOffsets[range_idx] &&
+ rko_offset <= LastOffsets[range_idx]) {
+ in_acquired_range = rd_true;
+ break;
+ }
+ }
+
+ if (in_acquired_range) {
+ /* Set ack type based on op type */
+ rd_kafka_msg_t *rkm = NULL;
+ if (rko->rko_type == RD_KAFKA_OP_FETCH) {
+ rkm = &rko->rko_u.fetch.rkm;
+ rkm->rkm_u.consumer.ack_type =
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED;
+ } else if (rko->rko_type == RD_KAFKA_OP_CONSUMER_ERR) {
+ rkm = &rko->rko_u.err.rkm;
+ rkm->rkm_u.consumer.ack_type =
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT;
+ }
+
+ /* Add to filtered messages list */
+ rd_list_add(filtered_msgs, rko);
+ } else {
+ /* Discard message not in any acquired range */
+ rd_kafka_op_destroy(rko);
+ }
+ }
+}
+
+
+/**
+ * @brief Comparator for sorting ops by offset.
+ *
+ * Note: rd_list_sort uses rd_list_cmp_trampoline which dereferences
+ * the pointers before calling this comparator, so we receive
+ * rd_kafka_op_t* directly, not rd_kafka_op_t**.
+ */
+static int rd_kafka_op_offset_cmp(const void *_a, const void *_b) {
+ const rd_kafka_op_t *a = (const rd_kafka_op_t *)_a;
+ const rd_kafka_op_t *b = (const rd_kafka_op_t *)_b;
+ int64_t off_a = rd_kafka_op_get_offset(a);
+ int64_t off_b = rd_kafka_op_get_offset(b);
+
+ return (off_a > off_b) - (off_a < off_b);
+}
+
+/**
+ * @brief Get partition from an op.
+ */
+static int32_t rd_kafka_op_get_partition(const rd_kafka_op_t *rko) {
+ if (rko->rko_type == RD_KAFKA_OP_FETCH)
+ return rko->rko_u.fetch.rkm.rkm_partition;
+ if (rko->rko_type == RD_KAFKA_OP_CONSUMER_ERR)
+ return rko->rko_u.err.rkm.rkm_partition;
+ return -1;
+}
+
+/**
+ * @brief Check if op matches the given partition.
+ */
+static rd_bool_t rd_kafka_op_matches_partition(const rd_kafka_op_t *rko,
+ int32_t partition) {
+ return rd_kafka_op_get_partition(rko) == partition;
+}
+
+/** List destructor for inflight_acks: destroy batch and its rktpar. */
+static void rd_kafka_share_ack_batches_destroy_cb(void *ptr) {
+ rd_kafka_share_ack_batches_destroy((rd_kafka_share_ack_batches_t *)ptr,
+ rd_true);
+}
+
+/**
+ * @brief Find the batch entry that contains the given offset.
+ *
+ * @returns The entry, or NULL if no entry contains the offset.
+ */
+static rd_kafka_share_ack_batch_entry_t *
+rd_kafka_share_find_entry_for_offset(rd_kafka_share_ack_batches_t *batches,
+ int64_t offset) {
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int ei;
+
+ RD_LIST_FOREACH(entry, &batches->entries, ei) {
+ if (offset >= entry->start_offset &&
+ offset <= entry->end_offset)
+ return entry;
+ }
+ return NULL;
+}
+
+/**
+ * @brief Get ack type from a message op (FETCH or CONSUMER_ERR).
+ */
+static rd_kafka_share_internal_acknowledgement_type
+rd_kafka_share_ack_type_from_msg_op(rd_kafka_op_t *msg_rko) {
+ rd_kafka_msg_t *rkm;
+
+ if (msg_rko->rko_type == RD_KAFKA_OP_FETCH)
+ rkm = &msg_rko->rko_u.fetch.rkm;
+ else
+ rkm = &msg_rko->rko_u.err.rkm;
+ return (rd_kafka_share_internal_acknowledgement_type)
+ rkm->rkm_u.consumer.ack_type;
+}
+
+/**
+ * @brief Build share fetch response RKO with messages and update inflight acks.
+ *
+ * All entry types are pre-initialized to GAP in the reply handler. This
+ * function overwrites ACQUIRED/REJECT for each offset that has a message
+ * (RD_KAFKA_OP_FETCH or RD_KAFKA_OP_CONSUMER_ERR), and adds those messages
+ * to the response. Moves inflight_acks batches to the response RKO.
+ *
+ * @param rkb Broker handle
+ * @param filtered_msgs List of filtered message ops (with ack types already
+ * set)
+ * @param inflight_acks List of rd_kafka_share_ack_batches_t* (ownership
+ * transferred to RKO)
+ *
+ * @returns New rko containing messages and inflight_acks, or NULL if empty
+ */
+static rd_kafka_op_t *
+rd_kafka_share_build_response_rko(rd_kafka_broker_t *rkb,
+ rd_list_t *filtered_msgs,
+ rd_list_t *inflight_acks) {
+
+ rd_kafka_op_t *response_rko;
+ rd_kafka_share_ack_batches_t *batches;
+ int32_t msg_cnt = 0;
+ int32_t inflight_acks_cnt = rd_list_cnt(inflight_acks);
+ int pi, i;
+ int total_msgs = rd_list_cnt(filtered_msgs);
+ int msg_start_idx = 0;
+ int64_t total_offsets = 0;
+
+ /* Create response rko */
+ response_rko = rd_kafka_op_new(RD_KAFKA_OP_SHARE_FETCH_RESPONSE);
+ response_rko->rko_rk = rkb->rkb_rk;
+
+ response_rko->rko_u.share_fetch_response.message_rkos =
+ rd_list_new(0, NULL);
+ response_rko->rko_u.share_fetch_response.inflight_acks =
+ rd_list_new(0, NULL);
+
+ /* Process each partition: set types for message offsets, add messages.
+ * Messages in filtered_msgs are grouped by partition in the same order
+ * as inflight_acks. Types are already GAP; we set ACQUIRED/REJECT only
+ * where we have a message. */
+ RD_LIST_FOREACH(batches, inflight_acks, pi) {
+
+ total_offsets += batches->response_msgs_count;
+
+ rd_list_t partition_msgs;
+ rd_list_init(&partition_msgs, 0, NULL);
+
+ while (msg_start_idx < total_msgs) {
+ rd_kafka_op_t *candidate =
+ rd_list_elem(filtered_msgs, msg_start_idx);
+ if (!rd_kafka_op_matches_partition(
+ candidate, batches->rktpar->partition))
+ break;
+ rd_list_add(&partition_msgs, candidate);
+ msg_start_idx++;
+ }
+
+ rd_list_sort(&partition_msgs, rd_kafka_op_offset_cmp);
+
+ int partition_msg_cnt = rd_list_cnt(&partition_msgs);
+ for (i = 0; i < partition_msg_cnt; i++) {
+ rd_kafka_op_t *msg_rko =
+ rd_list_elem(&partition_msgs, i);
+ int64_t offset = rd_kafka_op_get_offset(msg_rko);
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_kafka_share_find_entry_for_offset(batches,
+ offset);
+
+ if (unlikely(!entry)) {
+ rd_kafka_dbg(
+ rkb->rkb_rk, FETCH, "SHAREFETCH",
+ "No ack entry found for offset %" PRId64
+ " on [%" PRId32 "], skipping",
+ offset, batches->rktpar->partition);
+ continue;
+ }
+
+ entry->types[offset - entry->start_offset] =
+ rd_kafka_share_ack_type_from_msg_op(msg_rko);
+
+ if (msg_rko->rko_type == RD_KAFKA_OP_FETCH) {
+ rd_list_add(
+ response_rko->rko_u.share_fetch_response
+ .message_rkos,
+ msg_rko);
+ msg_cnt++;
+ } else {
+ /* RD_KAFKA_OP_CONSUMER_ERR: forward to
+ * consumer group queue */
+ rd_kafka_q_enq(rkb->rkb_rk->rk_cgrp->rkcg_q,
+ msg_rko);
+ }
+ }
+
+ rd_list_destroy(&partition_msgs);
+ rd_list_add(
+ response_rko->rko_u.share_fetch_response.inflight_acks,
+ batches);
+ }
+
+ rd_list_init(inflight_acks, 0, NULL);
+
+ if (msg_cnt == 0 && inflight_acks_cnt == 0) {
+ rd_kafka_op_destroy(response_rko);
+ return NULL;
+ }
+
+ rd_rkb_dbg(rkb, FETCH, "SHAREFETCH",
+ "Built share fetch response rko with %d messages, "
+ "%d gaps, and %d partitions",
+ msg_cnt, (int32_t)(total_offsets - msg_cnt),
+ inflight_acks_cnt);
+
+ return response_rko;
+}
+
+
+/**
+ * @brief Parse a partition from ShareFetch response and build inflight_acks.
+ *
+ * Creates rd_kafka_share_ack_batches_t directly with per-offset tracking.
+ * All offsets are initialized to ACQUIRED state; types will be updated
+ * later when matching with actual messages (GAP for missing, REJECT for
+ * errors).
+ *
+ * @param rkb Broker handle
+ * @param topic Topic name
+ * @param topic_id Topic UUID
+ * @param rkt Topic handle (possibly NULL)
+ * @param rkbuf Response buffer
+ * @param request Request buffer
+ * @param temp_appq Queue to forward filtered messages
+ * @param batches_out Output: inflight acks batches for this partition
+ *
+ * @returns Error code or RD_KAFKA_RESP_ERR_NO_ERROR on success
+ *
+ * TODO KIP-932: Check if we can reduce the number of args in this method.
+ * Can we remove the topic field from the args, since it is a derived value
+ * from rkt and initialize a topic variable inside this function?
+ */
+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_Uuid_t topic_id,
+ rd_kafka_topic_t *rkt /*possibly NULL*/,
+ rd_kafka_buf_t *rkbuf,
+ rd_kafka_buf_t *request,
+ rd_list_t *filtered_msgs,
+ rd_kafka_share_ack_batches_t *batches_out) {
+
+ 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 *FirstOffsets = NULL;
+ int64_t *LastOffsets = NULL;
+ int16_t *DeliveryCounts = NULL;
+ rd_kafka_q_t *temp_fetchq = rd_kafka_q_new(rkb->rkb_rk);
+ 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
+
+ if (AcknowledgementErrorCode != RD_KAFKA_RESP_ERR_NO_ERROR) {
+ rd_kafka_dbg(
+ rkb->rkb_rk, CGRP, "SHAREACK",
+ "ShareFetch response for %.*s [%" PRId32
+ "]: AcknowledgementError %" PRId16 " (%s)",
+ RD_KAFKAP_STR_PR(topic), PartitionId,
+ AcknowledgementErrorCode,
+ rd_kafka_err2str(AcknowledgementErrorCode));
+ printf("ShareFetch response for %.*s [%" PRId32
+ "]: AcknowledgementError %" PRId16 " (%s)\n",
+ RD_KAFKAP_STR_PR(topic), PartitionId,
+ AcknowledgementErrorCode,
+ rd_kafka_err2str(AcknowledgementErrorCode));
+ }
+
+ 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)) {
+ int64_t tmp_first, tmp_last;
+ int16_t tmp_delivery;
+ 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);
+ rd_kafka_buf_read_arraycnt(rkbuf, &AcquiredRecordsArrayCnt,
+ -1); // AcquiredRecordsArrayCnt
+ for (i = 0; i < AcquiredRecordsArrayCnt; i++) {
+ rd_kafka_buf_read_i64(rkbuf,
+ &tmp_first); // FirstOffset
+ rd_kafka_buf_read_i64(rkbuf, &tmp_last); // LastOffset
+ rd_kafka_buf_read_i16(rkbuf,
+ &tmp_delivery); // DeliveryCount
+ rd_kafka_buf_skip_tags(rkbuf); // AcquiredRecords tags
+ }
+ /* Initialize batches_out with NULL rktpar for unknown topic */
+ batches_out->rktpar = NULL;
+ rd_kafka_q_destroy(temp_fetchq);
+ rd_kafka_buf_skip_tags(rkbuf);
+ 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;
+
+ /**
+ * TODO KIP-932: Currently for CRC error, we are sending reject for only
+ * 1 offset. We should send reject for all offsets in the range.
+ */
+ 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
+ */
+ err = rd_kafka_share_msgset_parse(rkbuf, rktp, NULL, &tver,
+ temp_fetchq);
+
+ 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
+ "] : AcquiredRecordsArrayCnt: %d",
+ RD_KAFKAP_STR_PR(topic), PartitionId,
+ AcquiredRecordsArrayCnt);
+
+ rd_kafka_topic_partition_private_t *parpriv;
+ char *topic_str;
+
+ /* Allocate and initialize the topic partition */
+ topic_str = RD_KAFKAP_STR_DUP(topic);
+ batches_out->rktpar =
+ rd_kafka_topic_partition_new(topic_str, PartitionId);
+ rd_free(topic_str);
+ batches_out->rktpar->offset = RD_KAFKA_OFFSET_INVALID;
+
+ /* Allocate and fill the private structure */
+ parpriv = rd_kafka_topic_partition_private_new();
+ parpriv->rktp = rd_kafka_toppar_keep(rktp);
+ parpriv->topic_id = topic_id;
+ batches_out->rktpar->_private = parpriv;
+
+ batches_out->response_leader_id = CurrentLeader.LeaderId;
+ batches_out->response_leader_epoch = CurrentLeader.LeaderEpoch;
+ batches_out->response_msgs_count = 0;
+ rd_list_init(&batches_out->entries, AcquiredRecordsArrayCnt, NULL);
+
+ if (AcquiredRecordsArrayCnt > 0) {
+ FirstOffsets =
+ rd_malloc(sizeof(*FirstOffsets) * AcquiredRecordsArrayCnt);
+ LastOffsets =
+ rd_malloc(sizeof(*LastOffsets) * AcquiredRecordsArrayCnt);
+ DeliveryCounts = rd_malloc(sizeof(*DeliveryCounts) *
+ AcquiredRecordsArrayCnt);
+
+ for (i = 0; i < AcquiredRecordsArrayCnt; i++) {
+ rd_kafka_buf_read_i64(rkbuf, &FirstOffsets[i]);
+ rd_kafka_buf_read_i64(rkbuf, &LastOffsets[i]);
+ rd_kafka_buf_read_i16(rkbuf, &DeliveryCounts[i]);
+ rd_kafka_buf_skip_tags(rkbuf);
+
+ int64_t size = LastOffsets[i] - FirstOffsets[i] + 1;
+
+ rd_rkb_dbg(rkb, FETCH, "SHAREFETCH",
+ "%.*s [%" PRId32
+ "]: Acquired Records from offset %" PRId64
+ " to %" PRId64 ", DeliveryCount %" PRId16,
+ RD_KAFKAP_STR_PR(topic), PartitionId,
+ FirstOffsets[i], LastOffsets[i],
+ DeliveryCounts[i]);
+
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_kafka_share_ack_batch_entry_new(
+ FirstOffsets[i], LastOffsets[i], (int32_t)size);
+ entry->delivery_count = DeliveryCounts[i];
+
+ /* Initialize all offsets to GAP; build_response_rko
+ * will set ACQUIRED/REJECT for offsets that have a
+ * message. */
+ for (int64_t j = 0; j < size; j++) {
+ entry->types[j] =
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP;
+ }
+
+ rd_list_add(&batches_out->entries, entry);
+ batches_out->response_msgs_count += (int32_t)size;
+ }
+
+ /* Filter and forward messages in acquired ranges */
+ rd_kafka_share_filter_msg_from_acq_records(
+ temp_fetchq, filtered_msgs, AcquiredRecordsArrayCnt,
+ FirstOffsets, LastOffsets);
+
+ rd_free(FirstOffsets);
+ rd_free(LastOffsets);
+ rd_free(DeliveryCounts);
+ } else {
+ /* No acquired ranges: drop everything */
+ rd_kafka_op_t *rko;
+ while ((rko = rd_kafka_q_pop(temp_fetchq, RD_POLL_NOWAIT, 0)))
+ rd_kafka_op_destroy(rko);
+ }
+
+ rd_kafka_q_destroy_owner(temp_fetchq);
+
+ rd_kafka_buf_skip_tags(rkbuf); // Partition tags
+
+ goto done;
+
+err_parse:
+ return rkbuf->rkbuf_err;
+
+done:
+ 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_list_t filtered_msgs;
+ rd_list_t inflight_acks;
+ rd_bool_t has_acquired_records = rd_false;
+
+ rd_list_init(&filtered_msgs, 0, NULL);
+
+ 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));
+ rd_list_destroy(&filtered_msgs);
+ return ErrorCode;
+ }
+
+ rd_kafka_buf_read_i32(rkbuf, &AcquisitionLockTimeoutMs);
+
+ rd_kafka_buf_read_arraycnt(rkbuf, &TopicArrayCnt, RD_KAFKAP_TOPICS_MAX);
+
+ /* Initialize inflight_acks list (destructor frees rktpar) */
+ rd_list_init(&inflight_acks, 0, rd_kafka_share_ack_batches_destroy_cb);
+
+ 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++) {
+ rd_kafka_share_ack_batches_t *batches =
+ rd_kafka_share_ack_batches_new();
+
+ if (rd_kafka_share_fetch_reply_handle_partition(
+ rkb, &topic, topic_id, rkt, rkbuf, request,
+ &filtered_msgs, batches)) {
+ rd_kafka_share_ack_batches_destroy(batches,
+ rd_true);
+ goto err_parse;
+ }
+
+ /* Skip unknown topics - don't add to inflight_acks */
+ if (batches->rktpar == NULL) {
+ rd_kafka_share_ack_batches_destroy(batches,
+ rd_true);
+ continue;
+ }
+
+ if (rd_list_cnt(&batches->entries) > 0)
+ has_acquired_records = rd_true;
+
+ rd_list_add(&inflight_acks, batches);
+ }
+
+ if (rkt) {
+ rd_kafka_topic_destroy0(rkt);
+ rkt = NULL;
+ }
+ /* Topic Tags */
+ rd_kafka_buf_skip_tags(rkbuf);
+ }
+
+ rd_kafka_buf_read_NodeEndpoints(rkbuf, &NodeEndpoints);
+
+ /* Build response rko with messages and inflight_acks */
+ rd_kafka_op_t *response_rko = rd_kafka_share_build_response_rko(
+ rkb, &filtered_msgs, &inflight_acks);
+
+ if (response_rko)
+ rd_kafka_q_enq(rkb->rkb_rk->rk_cgrp->rkcg_q, response_rko);
+
+ rd_list_destroy(&filtered_msgs);
+
+ /* 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();
+ }
+
+ /* Signal main thread whether records were acquired */
+ rd_kafka_op_t *rko_orig = request->rkbuf_opaque;
+ if (rko_orig)
+ rko_orig->rko_u.share_fetch.records_fetched =
+ has_acquired_records;
+
+ RD_IF_FREE(NodeEndpoints.NodeEndpoints, rd_free);
+ RD_IF_FREE(rkt, rd_kafka_topic_destroy0);
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+
+err_parse:
+ /* Free inflight_acks list on error (destructor handles cleanup) */
+ rd_list_destroy(&inflight_acks);
+ rd_list_destroy(&filtered_msgs);
+
+ 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_toppars_list(
+ rd_kafka_broker_t *rkb,
+ rd_list_t **request_toppars_ptr,
+ rd_list_t **toppars_to_remove_from_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_from = *toppars_to_remove_from_ptr;
+
+ if (request_toppars == NULL || rd_list_cnt(request_toppars) == 0)
+ return;
+
+ rd_kafka_dbg(
+ rkb->rkb_rk, FETCH, "SHAREFETCH",
+ "%d toppars being %s the session:", rd_list_cnt(request_toppars),
+ add ? "added to" : "removed from");
+
+ RD_LIST_FOREACH(rktp, request_toppars, i) {
+ rd_kafka_dbg(
+ rkb->rkb_rk, FETCH, "SHAREFETCH", " %s [%" PRId32 "]",
+ rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition);
+ rd_kafka_broker_session_update_toppars_in_session(rkb, rktp,
+ add);
+ if (toppars_to_remove_from) {
+ removed_rktp =
+ rd_list_remove(toppars_to_remove_from, rktp);
+ if (removed_rktp) {
+ rd_kafka_toppar_destroy(
+ removed_rktp); /* from partitions list */
+ if (rd_list_empty(toppars_to_remove_from)) {
+ rd_list_destroy(toppars_to_remove_from);
+ toppars_to_remove_from = NULL;
+ *toppars_to_remove_from_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;
+
+ /**
+ * TODO KIP-932: Improve this handling with Error handling and leave
+ * flow.
+ */
+ 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);
+
+ rd_kafka_broker_session_update(rkb);
+
+ /* TODO: Check if this error handling is required here or in the main
+ * thread. */
+ 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;
+ }
+
+ /*
+ * TODO KIP-932: Check if this needed or not. If yes, check the
+ * working.
+ */
+ rd_kafka_broker_fetch_backoff(rkb, err);
+ /* FALLTHRU */
+ }
+
+ if (rko_orig)
+ rd_kafka_op_reply(rko_orig, err);
+ rkb->rkb_fetching = 0;
+}
/**
* @broker FetchResponse handling.
@@ -959,6 +1842,559 @@ 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_add,
+ rd_list_t *toppars_to_forget,
+ 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_add_cnt =
+ toppars_to_add ? rd_list_cnt(toppars_to_add) : 0;
+ int i;
+ rd_list_t *ack_details =
+ rko_orig ? rko_orig->rko_u.share_fetch.ack_details : NULL;
+ int ack_details_cnt = ack_details ? rd_list_cnt(ack_details) : 0;
+ /* TODO KIP-932: Ensure there is no intersection between toppars_to_add
+ * and ack_details. A toppar should not appear in both lists. */
+ int total_ack_entries = 0;
+ int toppars_to_forget_cnt =
+ toppars_to_forget ? rd_list_cnt(toppars_to_forget) : 0;
+ rd_bool_t is_fetching_messages = max_records > 0 ? rd_true : rd_false;
+ /* FirstOffset + LastOffset + AcknowledgementType per ack entry */
+ size_t acknowledgement_size = 8 + 8 + 1;
+
+ rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH",
+ "toppars_to_add_cnt=%d, ack_details_cnt=%d, "
+ "total_ack_entries=%d, toppars_to_forget_cnt=%d, "
+ "is_fetching_messages=%d",
+ toppars_to_add_cnt, ack_details_cnt, total_ack_entries,
+ toppars_to_forget_cnt, is_fetching_messages);
+
+ /* 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) for topics to add */
+ rkbuf_size += (toppars_to_add_cnt * (32 + 4));
+ /* M x (topic id + partition id) for ack details partitions +
+ * E x acknowledgement entries */
+
+ rkbuf_size += (ack_details_cnt * (32 + 4));
+ /* Count total ack entries across all ack_details batches */
+ if (ack_details) {
+ rd_kafka_share_ack_batches_t *batches;
+ int k;
+ RD_LIST_FOREACH(batches, ack_details, k) {
+ total_ack_entries += rd_list_cnt(&batches->entries);
+ }
+ }
+ rkbuf_size += (total_ack_entries * acknowledgement_size);
+
+ /* F x (topic id + partition id) for topics to forget */
+ rkbuf_size += (toppars_to_forget_cnt * (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);
+
+ /* TODO KIP-932: Ensure toppars_to_add and ack_details don't have
+ * common rktps. A toppar should only appear in one list.
+ * Also merge toppars_to_add and ack_details as both can have
+ * the same topic but different partitions. */
+
+ /* Write toppars_to_add: new toppars being added to session
+ * (no acknowledgements for newly added toppars) */
+ if (toppars_to_add) {
+ /* TODO KIP-932: This condition will cause partitions of same
+ topics to be inside single instance of the topic as
+ toppars_to_add 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. */
+ RD_LIST_FOREACH(rktp, toppars_to_add, i) {
+ 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);
+
+ /* No acknowledgements for newly added toppars */
+ rd_kafka_buf_write_arraycnt(rkbuf, 0);
+
+ /* Partition tags */
+ rd_kafka_buf_write_tags_empty(rkbuf);
+
+ rd_rkb_dbg(
+ rkb, FETCH, "SHAREFETCH",
+ "Share Fetch adding topic %.*s [%" PRId32 "]",
+ RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
+ rktp->rktp_partition);
+
+ cnt++;
+ }
+
+ /* Finalize last topic from toppars_to_add */
+ if (rkt_last != NULL) {
+ rd_kafka_buf_finalize_arraycnt(
+ rkbuf, of_PartitionArrayCnt, PartitionArrayCnt);
+ rd_kafka_buf_write_tags_empty(rkbuf);
+ }
+ }
+
+ /* Write ack_details: toppars with acknowledgement batches.
+ * Uses rktpar directly (no rktp/lock needed). */
+ if (ack_details) {
+ rd_kafka_Uuid_t *topic_id_last = NULL;
+ rd_kafka_share_ack_batches_t *batches;
+ rd_kafka_topic_partition_private_t *parpriv;
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int k, m, entries_cnt;
+
+ /* TODO KIP-932: This condition will cause partitions of same
+ topics to be inside single instance of the topic as
+ ack_details 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. */
+ PartitionArrayCnt = 0;
+
+ RD_LIST_FOREACH(batches, ack_details, k) {
+ /* TODO KIP-932: Ensure rktpar and rktp are in sync,
+ * leader has not changed in between, and rktp is
+ * present before reaching this point.
+ * Also ensure batches->entries are not empty. */
+ /* _private should always be present here as it was
+ * set when building the ack batches from the
+ * inflight map. */
+ parpriv =
+ (rd_kafka_topic_partition_private_t *)
+ batches->rktpar->_private;
+ rd_dassert(parpriv != NULL);
+
+ if (topic_id_last == NULL ||
+ rd_kafka_Uuid_cmp(*topic_id_last,
+ parpriv->topic_id) != 0) {
+ if (topic_id_last != NULL) {
+ /* Update PartitionArrayCnt */
+ rd_kafka_buf_finalize_arraycnt(
+ rkbuf, of_PartitionArrayCnt,
+ PartitionArrayCnt);
+ /* Topic tags */
+ rd_kafka_buf_write_tags_empty(rkbuf);
+ }
+
+ /* Topic ID */
+ rd_kafka_buf_write_uuid(rkbuf,
+ &parpriv->topic_id);
+
+ TopicArrayCnt++;
+ topic_id_last = &parpriv->topic_id;
+ /* Partition count */
+ of_PartitionArrayCnt =
+ rd_kafka_buf_write_arraycnt_pos(rkbuf);
+ PartitionArrayCnt = 0;
+ }
+
+ PartitionArrayCnt++;
+
+ /* Partition */
+ rd_kafka_buf_write_i32(rkbuf,
+ batches->rktpar->partition);
+
+ /* Write acknowledgement batches */
+ entries_cnt = rd_list_cnt(&batches->entries);
+ rd_kafka_buf_write_arraycnt(rkbuf, entries_cnt);
+
+ RD_LIST_FOREACH(entry, &batches->entries, m) {
+ /* FirstOffset */
+ rd_kafka_buf_write_i64(rkbuf,
+ entry->start_offset);
+ /* LastOffset */
+ rd_kafka_buf_write_i64(rkbuf,
+ entry->end_offset);
+ /* AcknowledgeTypes */
+ rd_kafka_buf_write_arraycnt(rkbuf, 1);
+ rd_kafka_buf_write_i8(
+ rkbuf, (int8_t)entry->types[0]);
+ /* Acknowledgement tags */
+ rd_kafka_buf_write_tags_empty(rkbuf);
+ }
+
+ /* Partition tags */
+ rd_kafka_buf_write_tags_empty(rkbuf);
+
+ rd_rkb_dbg(rkb, FETCH, "SHAREFETCH",
+ "Share Fetch ack for topic %s [%" PRId32
+ "] with %d entries",
+ batches->rktpar->topic,
+ batches->rktpar->partition, entries_cnt);
+
+ cnt++;
+ }
+
+ /* Finalize last topic from ack_details */
+ if (topic_id_last != NULL) {
+ rd_kafka_buf_finalize_arraycnt(
+ rkbuf, of_PartitionArrayCnt, PartitionArrayCnt);
+ rd_kafka_buf_write_tags_empty(rkbuf);
+ }
+ }
+
+ rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH",
+ "Share Fetch Request with %d toppars on %d topics", cnt,
+ TopicArrayCnt);
+
+ /* Update TopicArrayCnt */
+ rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, TopicArrayCnt);
+
+ if (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",
+ toppars_to_forget_cnt);
+ 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_forget 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);
+
+ /* Copy toppars_to_add/toppars_to_forget into adding/forgetting lists
+ * just before sending. On response, session_update() will move
+ * adding_toppars into toppars_in_session and remove from
+ * toppars_to_add. */
+ 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);
+
+ 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 */
+
+ return rd_list_new(0, NULL);
+}
+
+
+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);
+ rd_rkb_dbg(rkb, BROKER, "SHAREFETCH",
+ "%s [%" PRId32
+ "]: removed from ShareFetch session on clear",
+ rktp->rktp_rkt->rkt_topic->str,
+ rktp->rktp_partition);
+ rd_kafka_toppar_destroy(rktp); // from session list
+ }
+ 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_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 */
+ 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;
+ rd_list_t *ack_details = rko_orig->rko_u.share_fetch.ack_details;
+ rd_bool_t has_ack_details =
+ ack_details && rd_list_cnt(ack_details) > 0;
+
+ 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 = rkb->rkb_rk->rk_conf.share.max_poll_records;
+ }
+
+ if (rkb->rkb_share_fetch_session.toppars_to_add ||
+ rkb->rkb_share_fetch_session.toppars_to_forget ||
+ rko_orig->rko_u.share_fetch.should_fetch || has_ack_details) {
+ rd_kafka_dbg(rkb->rkb_rk, FETCH, "SHAREFETCH",
+ "Sending Share Fetch Request with%s%s%s%s",
+ has_ack_details ? " acknowledgements," : "",
+ rkb->rkb_share_fetch_session.toppars_to_add
+ ? " new topics,"
+ : "",
+ rkb->rkb_share_fetch_session.toppars_to_forget
+ ? " forgotten toppars,"
+ : "",
+ rko_orig->rko_u.share_fetch.should_fetch
+ ? " fetching messages"
+ : "");
+ } else {
+ rd_kafka_dbg(
+ rkb->rkb_rk, FETCH, "SHAREFETCH",
+ "Not sending Share Fetch Request since there are no "
+ "new topics to add, acknowledgements, forgotten toppars"
+ " or messages to fetch");
+ rd_kafka_op_reply(rko_orig, RD_KAFKA_RESP_ERR__NOOP);
+ return;
+ }
+
+ 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,
+ max_records, /* TODO KIP-932: Check if this is correct for batch
+ size or not */
+ rkb->rkb_share_fetch_session
+ .toppars_to_add, /* toppars to add to session */
+ rkb->rkb_share_fetch_session
+ .toppars_to_forget, /* forgetting toppars */
+ rko_orig, /* rko (carries ack_details) */
+ now);
+}
+
/**
* @brief Build and send a Fetch request message for all underflowed toppars
* for a specific broker.
@@ -1078,6 +2514,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..be192e1f05 100644
--- a/src/rdkafka_fetcher.h
+++ b/src/rdkafka_fetcher.h
@@ -40,5 +40,18 @@ 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);
+int64_t rd_kafka_op_get_offset(const rd_kafka_op_t *rko);
+void rd_kafka_share_filter_msg_from_acq_records(rd_kafka_q_t *temp_fetchq,
+ rd_list_t *filtered_msgs,
+ int32_t AcquiredRecordsArrayCnt,
+ const int64_t *FirstOffsets,
+ const int64_t *LastOffsets);
+
#endif /* _RDKAFKA_FETCHER_H_ */
diff --git a/src/rdkafka_int.h b/src/rdkafka_int.h
index d9986b320d..c056a63e49 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:
@@ -331,6 +334,13 @@ struct rd_kafka_s {
struct rd_kafka_cgrp_s *rk_cgrp;
+ rd_kafka_share_t
+ *rk_rkshare; /**< Share consumer handle (if share
+ * consumer), NULL otherwise.
+ * Set when rd_kafka_share_consumer_new()
+ * is called, used to access inflight
+ * acks for building ack batches. */
+
rd_kafka_conf_t rk_conf;
rd_kafka_q_t *rk_logq; /* Log queue if `log.queue` set */
char rk_name[128];
@@ -769,6 +779,42 @@ struct rd_kafka_s {
} rk_mock;
};
+/**
+ * @brief Map type for tracking inflight share acknowledgements.
+ * Key: topic-partition, Value: ack batches for that partition.
+ */
+typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
+ rd_kafka_share_ack_batches_t *)
+ rd_kafka_share_inflight_map_t;
+
+struct rd_kafka_share_s {
+ rd_kafka_t *rkshare_rk; /**< The shared rd_kafka_t instance */
+
+ /**
+ * TODO KIP-932: Decide whether we want to keep this here
+ * or move it in rd_kafka_t.
+ */
+ /** Map of inflight acknowledgements per topic-partition.
+ * Key: rd_kafka_topic_partition_t*
+ * Value: rd_kafka_share_ack_batches_t*
+ */
+ /**
+ * TODO KIP-932: Decide on cleanup of this map,
+ * new colleated entries are created when we will be sending
+ * the contents of this map with Commit/Next Poll
+ * We would have to cleanup this map then.
+ */
+ rd_kafka_share_inflight_map_t rkshare_inflight_acks;
+
+ /** Total number of unacknowledged messages across all partitions. */
+ int64_t rkshare_unacked_cnt;
+
+ /** Set when a FANOUT with fetch_more_records=true is sent.
+ * Reset when records are received in the consumer queue.
+ * Prevents duplicate fetch FANOUTs while one is in flight. */
+ rd_bool_t rkshare_fetch_more_records_requested;
+};
+
#define rd_kafka_wrlock(rk) rwlock_wrlock(&(rk)->rk_lock)
#define rd_kafka_rdlock(rk) rwlock_rdlock(&(rk)->rk_lock)
#define rd_kafka_rdunlock(rk) rwlock_rdunlock(&(rk)->rk_lock)
@@ -1258,4 +1304,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_mock.c b/src/rdkafka_mock.c
index cdc0445f28..f3d3e0b088 100644
--- a/src/rdkafka_mock.c
+++ b/src/rdkafka_mock.c
@@ -2708,6 +2708,7 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) {
rd_kafka_mock_broker_t *mrkb;
rd_kafka_mock_cgrp_classic_t *mcgrp_classic;
rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer;
+ rd_kafka_mock_sharegroup_t *mshgrp;
rd_kafka_mock_coord_t *mcoord;
rd_kafka_mock_error_stack_t *errstack;
thrd_t dummy_rkb_thread;
@@ -2726,6 +2727,9 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) {
while ((mcgrp_consumer = TAILQ_FIRST(&mcluster->cgrps_consumer)))
rd_kafka_mock_cgrp_consumer_destroy(mcgrp_consumer);
+ while ((mshgrp = TAILQ_FIRST(&mcluster->sharegrps)))
+ rd_kafka_mock_sharegroup_destroy(mshgrp);
+
while ((mcoord = TAILQ_FIRST(&mcluster->coords)))
rd_kafka_mock_coord_destroy(mcluster, mcoord);
@@ -2843,6 +2847,8 @@ rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new(rd_kafka_t *rk,
TAILQ_INIT(&mcluster->cgrps_consumer);
+ rd_kafka_mock_sharegrps_init(mcluster);
+
TAILQ_INIT(&mcluster->coords);
rd_list_init(&mcluster->pids, 16, rd_free);
diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h
index 0b81b312ef..c0f1b1def4 100644
--- a/src/rdkafka_mock.h
+++ b/src/rdkafka_mock.h
@@ -601,6 +601,88 @@ RD_EXPORT void rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(
rd_kafka_mock_cluster_t *mcluster,
int group_consumer_heartbeat_interval_ms);
+/**
+ * @brief Set the sharegroup session timeout in milliseconds.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param session_timeout_ms Session timeout in milliseconds.
+ */
+RD_EXPORT void
+rd_kafka_mock_sharegroup_set_session_timeout(rd_kafka_mock_cluster_t *mcluster,
+ int session_timeout_ms);
+
+/**
+ * @brief Set the sharegroup heartbeat interval in milliseconds.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param heartbeat_interval_ms Heartbeat interval in milliseconds.
+ */
+RD_EXPORT void rd_kafka_mock_sharegroup_set_heartbeat_interval(
+ rd_kafka_mock_cluster_t *mcluster,
+ int heartbeat_interval_ms);
+
+/**
+ * @brief Set the maximum number of times a record can be acquired
+ * before it is automatically archived (dead-lettered).
+ *
+ * Default is 5. Set to 0 for unlimited deliveries.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param max_attempts Maximum delivery attempts per record.
+ */
+RD_EXPORT void rd_kafka_mock_sharegroup_set_max_delivery_attempts(
+ rd_kafka_mock_cluster_t *mcluster,
+ int max_attempts);
+
+/**
+ * @brief Set the per-record lock duration in milliseconds.
+ *
+ * When a record is acquired, its lock expires after this duration.
+ * Default is 0, which falls back to the session timeout value.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param lock_duration_ms Lock duration in milliseconds.
+ */
+RD_EXPORT void rd_kafka_mock_sharegroup_set_record_lock_duration(
+ rd_kafka_mock_cluster_t *mcluster,
+ int lock_duration_ms);
+
+/**
+ * @brief Set a manual target assignment for a sharegroup.
+ *
+ * This allows tests to override the automatic partition assignment
+ * and manually specify which partitions each member should get.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param group_id The sharegroup ID.
+ * @param member_ids Array of member IDs (strings).
+ * @param assignments Array of partition lists (one per member).
+ * @param member_cnt Number of members (length of both arrays).
+ */
+RD_EXPORT void rd_kafka_mock_sharegroup_target_assignment(
+ rd_kafka_mock_cluster_t *mcluster,
+ const char *group_id,
+ const char **member_ids,
+ rd_kafka_topic_partition_list_t **assignments,
+ size_t member_cnt);
+
+/**
+ * @brief Retrieve the member IDs from a sharegroup.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param group_id The sharegroup ID.
+ * @param member_ids_out Output array of member IDs. Caller must free each
+ * string with rd_free() and the array itself.
+ * @param member_cnt_out Output count of members.
+ *
+ * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success,
+ * RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND if sharegroup not found.
+ */
+RD_EXPORT rd_kafka_resp_err_t
+rd_kafka_mock_sharegroup_get_member_ids(rd_kafka_mock_cluster_t *mcluster,
+ const char *group_id,
+ char ***member_ids_out,
+ size_t *member_cnt_out);
/**@}*/
diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c
index 0c75e003e5..88e81210ec 100644
--- a/src/rdkafka_mock_cgrp.c
+++ b/src/rdkafka_mock_cgrp.c
@@ -35,6 +35,7 @@
#include "rdkafka_int.h"
#include "rdbuf.h"
#include "rdkafka_mock_int.h"
+#include "rdkafka_mock_group_common.h"
static const char *rd_kafka_mock_cgrp_classic_state_names[] = {
@@ -72,10 +73,9 @@ rd_kafka_mock_cgrp_classic_set_state(rd_kafka_mock_cgrp_classic_t *mcgrp,
void rd_kafka_mock_cgrp_classic_member_active(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member) {
- rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
- "Marking mock consumer group member %s as active",
- member->id);
- member->ts_last_activity = rd_clock();
+ rd_kafka_mock_group_member_mark_active(mcgrp->cluster->rk, "classic",
+ member->id,
+ &member->ts_last_activity);
}
@@ -537,13 +537,8 @@ static void rd_kafka_mock_cgrp_classic_member_destroy(
rd_kafka_mock_cgrp_classic_member_t *rd_kafka_mock_cgrp_classic_member_find(
const rd_kafka_mock_cgrp_classic_t *mcgrp,
const rd_kafkap_str_t *MemberId) {
- const rd_kafka_mock_cgrp_classic_member_t *member;
- TAILQ_FOREACH(member, &mcgrp->members, link) {
- if (!rd_kafkap_str_cmp_str(MemberId, member->id))
- return (rd_kafka_mock_cgrp_classic_member_t *)member;
- }
-
- return NULL;
+ return RD_KAFKA_MOCK_MEMBER_FIND(&mcgrp->members, MemberId,
+ rd_kafka_mock_cgrp_classic_member_t);
}
@@ -659,13 +654,8 @@ void rd_kafka_mock_cgrp_classic_destroy(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_mock_cgrp_classic_t *
rd_kafka_mock_cgrp_classic_find(rd_kafka_mock_cluster_t *mcluster,
const rd_kafkap_str_t *GroupId) {
- rd_kafka_mock_cgrp_classic_t *mcgrp;
- TAILQ_FOREACH(mcgrp, &mcluster->cgrps_classic, link) {
- if (!rd_kafkap_str_cmp_str(GroupId, mcgrp->id))
- return mcgrp;
- }
-
- return NULL;
+ return RD_KAFKA_MOCK_GROUP_FIND(&mcluster->cgrps_classic, GroupId,
+ rd_kafka_mock_cgrp_classic_t);
}
@@ -1335,10 +1325,9 @@ rd_kafka_mock_cgrp_consumer_member_next_assignment(
void rd_kafka_mock_cgrp_consumer_member_active(
rd_kafka_mock_cgrp_consumer_t *mcgrp,
rd_kafka_mock_cgrp_consumer_member_t *member) {
- rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
- "Marking mock consumer group member %s as active",
- member->id);
- member->ts_last_activity = rd_clock();
+ rd_kafka_mock_group_member_mark_active(mcgrp->cluster->rk, "consumer",
+ member->id,
+ &member->ts_last_activity);
}
/**
@@ -1353,13 +1342,8 @@ void rd_kafka_mock_cgrp_consumer_member_active(
rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_find(
const rd_kafka_mock_cgrp_consumer_t *mcgrp,
const rd_kafkap_str_t *MemberId) {
- const rd_kafka_mock_cgrp_consumer_member_t *member;
- TAILQ_FOREACH(member, &mcgrp->members, link) {
- if (!rd_kafkap_str_cmp_str(MemberId, member->id))
- return (rd_kafka_mock_cgrp_consumer_member_t *)member;
- }
-
- return NULL;
+ return RD_KAFKA_MOCK_MEMBER_FIND(&mcgrp->members, MemberId,
+ rd_kafka_mock_cgrp_consumer_member_t);
}
/**
@@ -1700,13 +1684,8 @@ void rd_kafka_mock_cgrp_consumer_member_fenced(
rd_kafka_mock_cgrp_consumer_t *
rd_kafka_mock_cgrp_consumer_find(const rd_kafka_mock_cluster_t *mcluster,
const rd_kafkap_str_t *GroupId) {
- rd_kafka_mock_cgrp_consumer_t *mcgrp;
- TAILQ_FOREACH(mcgrp, &mcluster->cgrps_consumer, link) {
- if (!rd_kafkap_str_cmp_str(GroupId, mcgrp->id))
- return mcgrp;
- }
-
- return NULL;
+ return RD_KAFKA_MOCK_GROUP_FIND(&mcluster->cgrps_consumer, GroupId,
+ rd_kafka_mock_cgrp_consumer_t);
}
/**
@@ -1873,4 +1852,5 @@ void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster,
rd_kafka_mock_connection_t *mconn) {
rd_kafka_mock_cgrps_classic_connection_closed(mcluster, mconn);
rd_kafka_mock_cgrps_consumer_connection_closed(mcluster, mconn);
+ rd_kafka_mock_sharegrps_connection_closed(mcluster, mconn);
}
diff --git a/src/rdkafka_mock_group_common.h b/src/rdkafka_mock_group_common.h
new file mode 100644
index 0000000000..367fccab06
--- /dev/null
+++ b/src/rdkafka_mock_group_common.h
@@ -0,0 +1,98 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file rdkafka_mock_group_common.h
+ * @brief Common macros and inline functions shared between
+ * ConsumerGroupHeartbeat (KIP-848) and ShareGroupHeartbeat (KIP-932)
+ * mock broker implementations.
+ */
+
+#ifndef _RDKAFKA_MOCK_GROUP_COMMON_H_
+#define _RDKAFKA_MOCK_GROUP_COMMON_H_
+
+#include "rdkafka_int.h"
+
+/**
+ * @brief Generic group find by GroupId in a TAILQ.
+ *
+ * @param groups_head Pointer to TAILQ_HEAD of groups.
+ * @param GroupId rd_kafkap_str_t pointer to search for.
+ * @param group_type The group struct type (e.g., rd_kafka_mock_sharegroup_t).
+ *
+ * @returns Pointer to found group, or NULL if not found.
+ */
+#define RD_KAFKA_MOCK_GROUP_FIND(groups_head, GroupId, group_type) \
+ ({ \
+ group_type *_grp = NULL; \
+ TAILQ_FOREACH(_grp, groups_head, link) { \
+ if (!rd_kafkap_str_cmp_str(GroupId, _grp->id)) \
+ break; \
+ } \
+ _grp; \
+ })
+
+/**
+ * @brief Generic member find by MemberId in a TAILQ.
+ *
+ * @param members_head Pointer to TAILQ_HEAD of members.
+ * @param MemberId rd_kafkap_str_t pointer to search for.
+ * @param member_type The member struct type.
+ *
+ * @returns Pointer to found member, or NULL if not found.
+ */
+#define RD_KAFKA_MOCK_MEMBER_FIND(members_head, MemberId, member_type) \
+ ({ \
+ member_type *_member = NULL; \
+ TAILQ_FOREACH(_member, members_head, link) { \
+ if (!rd_kafkap_str_cmp_str(MemberId, _member->id)) \
+ break; \
+ } \
+ _member; \
+ })
+
+/**
+ * @brief Mark a group member as active by updating its last activity timestamp.
+ *
+ * @param rk rd_kafka_t handle for logging.
+ * @param group_type String describing the group type ("consumer" or "share").
+ * @param member_id Member ID string for logging.
+ * @param ts_last_activity Pointer to the member's last activity timestamp.
+ */
+static RD_INLINE RD_UNUSED void
+rd_kafka_mock_group_member_mark_active(rd_kafka_t *rk,
+ const char *group_type,
+ const char *member_id,
+ rd_ts_t *ts_last_activity) {
+ rd_kafka_dbg(rk, MOCK, "MOCK",
+ "Marking mock %s group member %s as active", group_type,
+ member_id);
+ *ts_last_activity = rd_clock();
+}
+
+#endif /* _RDKAFKA_MOCK_GROUP_COMMON_H_ */
diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c
index ad509ecceb..e48f32682f 100644
--- a/src/rdkafka_mock_handlers.c
+++ b/src/rdkafka_mock_handlers.c
@@ -1461,7 +1461,8 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn,
rd_kafka_buf_write_i32(resp, INT32_MIN);
}
- rd_kafka_buf_skip_tags(rkbuf);
+ if (rd_kafka_buf_read_remain(rkbuf) > 0)
+ rd_kafka_buf_skip_tags(rkbuf);
rd_kafka_buf_write_tags_empty(resp);
if (requested_topics)
@@ -2993,6 +2994,1021 @@ rd_kafka_mock_handle_ConsumerGroupHeartbeat(rd_kafka_mock_connection_t *mconn,
return -1;
}
+/**
+ * @brief Helper to write assignment TopicPartitions to ShareGroupHeartbeat
+ * response.
+ */
+static void rd_kafka_mock_handle_ShareGroupHeartbeat_write_TopicPartitions(
+ rd_kafka_buf_t *resp,
+ rd_kafka_topic_partition_list_t *assignment) {
+ const rd_kafka_topic_partition_field_t fields[] = {
+ RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
+ RD_KAFKA_TOPIC_PARTITION_FIELD_END};
+
+ rd_kafka_topic_partition_list_sort_by_topic_id(assignment);
+ rd_kafka_buf_write_topic_partitions(
+ resp, assignment, rd_false /* don't skip invalid offsets */,
+ rd_false /* any offset */, rd_true /* use topic id */,
+ rd_false /* don't use topic name */, fields);
+}
+
+/**
+ * @brief Handle ShareGroupHeartbeat request (API Key 76).
+ */
+static int
+rd_kafka_mock_handle_ShareGroupHeartbeat(rd_kafka_mock_connection_t *mconn,
+ rd_kafka_buf_t *rkbuf) {
+ const rd_bool_t log_decode_errors = rd_true;
+ rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster;
+ rd_kafka_buf_t *resp;
+ rd_kafkap_str_t GroupId, MemberId, RackId;
+ rd_kafkap_str_t *SubscribedTopicNames = NULL;
+ int32_t MemberEpoch = 0, SubscribedTopicNamesCnt;
+ int32_t i;
+ rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
+ rd_kafka_mock_sharegroup_t *mshgrp = NULL;
+ rd_kafka_mock_sharegroup_member_t *member = NULL;
+ rd_bool_t assignment_changed = rd_false;
+
+ resp = rd_kafka_mock_buf_new_response(rkbuf);
+
+ /* Inject Error */
+ err = rd_kafka_mock_next_request_error(mconn, resp);
+ if (err)
+ goto build_response;
+
+ /* GroupId */
+ rd_kafka_buf_read_str(rkbuf, &GroupId);
+
+ /* Coordinator check */
+ {
+ rd_kafka_mock_broker_t *mrkb;
+
+ mrkb = rd_kafka_mock_cluster_get_coord(
+ mcluster, RD_KAFKA_COORD_GROUP, &GroupId);
+
+ if (!mrkb)
+ err = RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE;
+ else if (mrkb != mconn->broker)
+ err = RD_KAFKA_RESP_ERR_NOT_COORDINATOR;
+ }
+
+ if (err)
+ goto build_response;
+
+ /* MemberId */
+ rd_kafka_buf_read_str(rkbuf, &MemberId);
+
+ /* MemberEpoch */
+ rd_kafka_buf_read_i32(rkbuf, &MemberEpoch);
+
+ /* RackId (nullable) */
+ rd_kafka_buf_read_str(rkbuf, &RackId);
+
+ /* SubscribedTopicNames array (nullable) */
+ rd_kafka_buf_read_arraycnt(rkbuf, &SubscribedTopicNamesCnt,
+ RD_KAFKAP_TOPICS_MAX);
+ if (SubscribedTopicNamesCnt >= 0) {
+ SubscribedTopicNames = rd_calloc(
+ SubscribedTopicNamesCnt > 0 ? SubscribedTopicNamesCnt : 1,
+ sizeof(rd_kafkap_str_t));
+ for (i = 0; i < SubscribedTopicNamesCnt; i++) {
+ rd_kafka_buf_read_str(rkbuf, &SubscribedTopicNames[i]);
+ }
+ }
+
+ {
+ mtx_lock(&mcluster->lock);
+
+ mshgrp = rd_kafka_mock_sharegroup_get(mcluster, &GroupId);
+
+ if (MemberEpoch == -1) {
+ /* LEAVE: Member wants to leave */
+ member = rd_kafka_mock_sharegroup_member_find(
+ mshgrp, &MemberId);
+ if (member) {
+ rd_kafka_mock_sharegroup_member_destroy(mshgrp,
+ member);
+ member = NULL;
+ assignment_changed = rd_true;
+ }
+
+ } else if (MemberEpoch == 0) {
+ /* JOIN: New member wants to join */
+ member = rd_kafka_mock_sharegroup_member_get(
+ mshgrp, &MemberId, MemberEpoch, mconn);
+
+ if (member) {
+ if (rd_kafka_mock_sharegroup_member_subscribed_topic_names_set(
+ member, SubscribedTopicNames,
+ SubscribedTopicNamesCnt)) {
+ assignment_changed = rd_true;
+ } else {
+ /* New member always triggers
+ * recalculation */
+ assignment_changed = rd_true;
+ }
+ MemberEpoch = member->member_epoch;
+ } else {
+ err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
+ }
+
+ } else {
+ /* HEARTBEAT: Existing member heartbeat */
+ member = rd_kafka_mock_sharegroup_member_find(
+ mshgrp, &MemberId);
+ if (!member) {
+ err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
+ } else if (MemberEpoch > member->member_epoch) {
+ /* Client epoch is ahead of server - indicates
+ * a bug or stale coordinator. */
+ err = RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH;
+ } else if (MemberEpoch < member->member_epoch) {
+ /* Client epoch is behind. Allow if it matches
+ * the previous epoch (response with bumped
+ * epoch may have been lost). Otherwise fence.
+ */
+ if (MemberEpoch !=
+ member->previous_member_epoch) {
+ err =
+ RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH;
+ } else {
+ /* Accept previous epoch - client is
+ * catching up */
+ member->conn = mconn;
+ MemberEpoch = member->member_epoch;
+ rd_kafka_mock_sharegroup_member_active(
+ mshgrp, member);
+ }
+ } else {
+ /* Epoch matches - normal heartbeat */
+ /* Check for subscription changes */
+ if (SubscribedTopicNamesCnt >= 0 &&
+ rd_kafka_mock_sharegroup_member_subscribed_topic_names_set(
+ member, SubscribedTopicNames,
+ SubscribedTopicNamesCnt)) {
+ assignment_changed = rd_true;
+ }
+ member->conn = mconn;
+ MemberEpoch = member->member_epoch;
+ rd_kafka_mock_sharegroup_member_active(mshgrp,
+ member);
+ }
+ }
+
+ /* Recalculate assignments if needed */
+ if (assignment_changed && mshgrp->member_cnt > 0) {
+ rd_kafka_mock_sharegroup_assignment_recalculate(mshgrp);
+ if (member)
+ MemberEpoch = member->member_epoch;
+ }
+
+ mtx_unlock(&mcluster->lock);
+ }
+
+build_response:
+
+ /* ThrottleTimeMs */
+ rd_kafka_buf_write_i32(resp, 0);
+
+ /* ErrorCode */
+ rd_kafka_buf_write_i16(resp, err);
+
+ /* ErrorMessage */
+ if (err)
+ rd_kafka_buf_write_str(resp, rd_kafka_err2str(err), -1);
+ else
+ rd_kafka_buf_write_str(resp, NULL, -1);
+
+ /* MemberId */
+ if (!err && member)
+ rd_kafka_buf_write_str(resp, member->id, -1);
+ else
+ rd_kafka_buf_write_str(resp, NULL, -1);
+
+ /* MemberEpoch */
+ rd_kafka_buf_write_i32(resp, MemberEpoch);
+
+ /* HeartbeatIntervalMs */
+ if (mshgrp)
+ rd_kafka_buf_write_i32(resp, mshgrp->heartbeat_interval_ms);
+ else
+ rd_kafka_buf_write_i32(resp, 5000);
+
+ /* Assignment */
+ if (!err && member && member->assignment) {
+ /* Send assignment even if empty (cnt == 0).
+ * Null (-1) means "no change", while an empty assignment
+ * means "you have 0 partitions". */
+ rd_kafka_buf_write_i8(resp, 1);
+ rd_kafka_mock_handle_ShareGroupHeartbeat_write_TopicPartitions(
+ resp, member->assignment);
+ rd_kafka_buf_write_tags_empty(resp);
+ } else {
+ rd_kafka_buf_write_i8(resp, -1);
+ }
+
+ rd_kafka_buf_write_tags_empty(resp);
+
+ rd_kafka_mock_connection_send_response(mconn, resp);
+
+ RD_IF_FREE(SubscribedTopicNames, rd_free);
+ return 0;
+
+err_parse:
+ RD_IF_FREE(SubscribedTopicNames, rd_free);
+ rd_kafka_buf_destroy(resp);
+ return -1;
+}
+
+/**
+ * @brief Handle ShareFetch (KIP-932) - request parsing only.
+ */
+static rd_bool_t
+rd_kafka_mock_tplist_equal_by_id(rd_kafka_topic_partition_list_t *a,
+ rd_kafka_topic_partition_list_t *b) {
+ int i;
+
+ if (!a && !b)
+ return rd_true;
+ if (!a || !b)
+ return rd_false;
+ if (a->cnt != b->cnt)
+ return rd_false;
+
+ rd_kafka_topic_partition_list_sort_by_topic_id(a);
+ rd_kafka_topic_partition_list_sort_by_topic_id(b);
+
+ for (i = 0; i < a->cnt; i++) {
+ const rd_kafka_topic_partition_t *pa = &a->elems[i];
+ const rd_kafka_topic_partition_t *pb = &b->elems[i];
+ rd_kafka_Uuid_t ta = rd_kafka_topic_partition_get_topic_id(pa);
+ rd_kafka_Uuid_t tb = rd_kafka_topic_partition_get_topic_id(pb);
+
+ if (pa->partition != pb->partition)
+ return rd_false;
+ if (rd_kafka_Uuid_cmp(ta, tb) != 0)
+ return rd_false;
+ }
+
+ return rd_true;
+}
+
+static rd_kafka_mock_sgrp_partmeta_t *
+rd_kafka_mock_sgrp_partmeta_find(rd_kafka_mock_sharegroup_t *sgrp,
+ rd_kafka_Uuid_t topic_id,
+ int32_t partition) {
+ rd_kafka_mock_sgrp_partmeta_t *pmeta;
+
+ TAILQ_FOREACH(pmeta, &sgrp->partitions, link) {
+ if (pmeta->partition != partition)
+ continue;
+ if (!rd_kafka_Uuid_cmp(pmeta->topic_id, topic_id))
+ return pmeta;
+ }
+
+ return NULL;
+}
+
+static rd_kafka_mock_sgrp_partmeta_t *
+rd_kafka_mock_sgrp_partmeta_get(rd_kafka_mock_sharegroup_t *sgrp,
+ rd_kafka_Uuid_t topic_id,
+ int32_t partition,
+ const rd_kafka_mock_partition_t *mpart) {
+ rd_kafka_mock_sgrp_partmeta_t *pmeta;
+ int64_t log_start;
+ int64_t log_end;
+
+ pmeta = rd_kafka_mock_sgrp_partmeta_find(sgrp, topic_id, partition);
+ if (pmeta) {
+ log_start = mpart->start_offset;
+ log_end = mpart->end_offset;
+ if (log_start > pmeta->spso)
+ pmeta->spso = log_start;
+ if (log_end > log_start) {
+ int64_t new_speo = log_end - 1;
+ if (new_speo > pmeta->speo)
+ pmeta->speo = new_speo;
+ }
+ return pmeta;
+ }
+
+ pmeta = rd_calloc(1, sizeof(*pmeta));
+ pmeta->topic_id = topic_id;
+ pmeta->partition = partition;
+ pmeta->spso = mpart->start_offset;
+ if (mpart->end_offset > mpart->start_offset)
+ pmeta->speo = mpart->end_offset - 1;
+ else
+ pmeta->speo = mpart->start_offset - 1;
+ TAILQ_INIT(&pmeta->inflight);
+
+ TAILQ_INSERT_TAIL(&sgrp->partitions, pmeta, link);
+ sgrp->partition_cnt++;
+
+ return pmeta;
+}
+
+static rd_kafka_mock_sgrp_record_state_t *
+rd_kafka_mock_sgrp_record_state_find(rd_kafka_mock_sgrp_partmeta_t *pmeta,
+ int64_t offset) {
+ rd_kafka_mock_sgrp_record_state_t *state;
+
+ TAILQ_FOREACH(state, &pmeta->inflight, link) {
+ if (state->offset == offset)
+ return state;
+ }
+
+ return NULL;
+}
+
+static rd_kafka_mock_sgrp_record_state_t *
+rd_kafka_mock_sgrp_record_state_get(rd_kafka_mock_sgrp_partmeta_t *pmeta,
+ int64_t offset) {
+ rd_kafka_mock_sgrp_record_state_t *state;
+
+ state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset);
+ if (state)
+ return state;
+
+ state = rd_calloc(1, sizeof(*state));
+ state->offset = offset;
+ state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE;
+ TAILQ_INSERT_TAIL(&pmeta->inflight, state, link);
+ pmeta->inflight_cnt++;
+
+ return state;
+}
+
+static int32_t
+rd_kafka_mock_msgset_est_record_size(const rd_kafka_mock_msgset_t *mset) {
+ int64_t record_cnt = mset->last_offset - mset->first_offset + 1;
+ int32_t size;
+
+ if (record_cnt <= 0)
+ return 1;
+
+ size = (int32_t)(RD_KAFKAP_BYTES_LEN(&mset->bytes) / record_cnt);
+ if (size <= 0)
+ size = 1;
+
+ return size;
+}
+
+static void rd_kafka_mock_sgrp_acquire_available_offsets(
+ rd_kafka_mock_sgrp_partmeta_t *pmeta,
+ const rd_kafka_mock_partition_t *mpart,
+ const rd_kafkap_str_t *member_id,
+ rd_ts_t lock_expiry_ts,
+ int max_delivery_attempts,
+ int64_t *remaining_records,
+ int64_t *remaining_bytes,
+ int *acquired_cnt,
+ int64_t *acquired_bytes) {
+ int64_t offset;
+
+ for (offset = pmeta->spso; offset <= pmeta->speo; offset++) {
+ const rd_kafka_mock_msgset_t *mset;
+ rd_kafka_mock_sgrp_record_state_t *state;
+ int32_t est_size;
+
+ if (remaining_records && *remaining_records == 0)
+ break;
+ if (remaining_bytes && *remaining_bytes == 0)
+ break;
+
+ state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset);
+ if (state &&
+ state->state != RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE)
+ continue;
+
+ /* Check max delivery attempts: if the record has already
+ * been acquired (and released/expired) too many times,
+ * archive it instead of re-acquiring. */
+ if (max_delivery_attempts > 0 && state &&
+ state->delivery_count >= max_delivery_attempts) {
+ state->state = RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED;
+ RD_IF_FREE(state->owner_member_id, rd_free);
+ state->owner_member_id = NULL;
+ state->lock_expiry_ts = 0;
+ continue;
+ }
+
+ mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false);
+ if (!mset)
+ continue;
+
+ est_size = rd_kafka_mock_msgset_est_record_size(mset);
+ if (remaining_bytes && *remaining_bytes > 0 &&
+ est_size > *remaining_bytes)
+ break;
+
+ state = rd_kafka_mock_sgrp_record_state_get(pmeta, offset);
+ state->state = RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED;
+ state->lock_expiry_ts = lock_expiry_ts;
+ state->delivery_count++;
+ RD_IF_FREE(state->owner_member_id, rd_free);
+ state->owner_member_id = RD_KAFKAP_STR_DUP(member_id);
+
+ (*acquired_cnt)++;
+ *acquired_bytes += est_size;
+ if (remaining_records && *remaining_records > 0)
+ (*remaining_records)--;
+ if (remaining_bytes && *remaining_bytes > 0)
+ (*remaining_bytes) -= est_size;
+ }
+}
+
+static void rd_kafka_mock_sgrp_partmeta_prune_archived(
+ rd_kafka_mock_sgrp_partmeta_t *pmeta) {
+ rd_kafka_mock_sgrp_record_state_t *state, *tmp;
+
+ TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) {
+ if (state->state != RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED)
+ continue;
+ if (state->offset >= pmeta->spso)
+ continue;
+
+ TAILQ_REMOVE(&pmeta->inflight, state, link);
+ pmeta->inflight_cnt--;
+ rd_free(state->owner_member_id);
+ rd_free(state);
+ }
+}
+
+/**
+ * @brief Write all acquired record batches for the given partition and member
+ * into the response buffer as a single Records field (compact bytes
+ * containing concatenated RecordBatches).
+ *
+ * @returns the total number of record data bytes written (0 if none).
+ */
+static size_t rd_kafka_mock_sgrp_write_acquired_records(
+ rd_kafka_buf_t *resp,
+ rd_kafka_mock_sgrp_partmeta_t *pmeta,
+ const rd_kafka_mock_partition_t *mpart,
+ const rd_kafkap_str_t *member_id,
+ rd_ts_t now) {
+ rd_list_t msgsets;
+ int64_t offset;
+ size_t total_len = 0;
+ int i;
+
+ rd_list_init(&msgsets, 16, NULL); /* no free_cb: borrowed ptrs */
+
+ /* Collect unique msgsets containing acquired records for this member */
+ for (offset = pmeta->spso; offset <= pmeta->speo; offset++) {
+ rd_kafka_mock_sgrp_record_state_t *state =
+ rd_kafka_mock_sgrp_record_state_find(pmeta, offset);
+ const rd_kafka_mock_msgset_t *mset;
+
+ if (!state ||
+ state->state != RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED)
+ continue;
+
+ if (state->lock_expiry_ts && state->lock_expiry_ts <= now)
+ continue;
+
+ if (rd_kafkap_str_cmp_str(member_id, state->owner_member_id))
+ continue;
+
+ mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false);
+ if (!mset)
+ continue;
+
+ /* Deduplicate: multiple offsets may fall in the same batch */
+ if (rd_list_find(&msgsets, mset, rd_list_cmp_ptr))
+ continue;
+
+ rd_list_add(&msgsets, (void *)mset);
+ total_len += RD_KAFKAP_BYTES_LEN(&mset->bytes);
+ }
+
+ if (rd_list_cnt(&msgsets) == 0) {
+ /* No acquired records: write NULL compact bytes */
+ rd_list_destroy(&msgsets);
+ rd_kafka_buf_write_kbytes(resp, NULL);
+ return 0;
+ }
+
+ /* Write compact bytes length prefix (N+1 encoding) */
+ rd_kafka_buf_write_uvarint(resp, (uint64_t)(total_len + 1));
+
+ /* Write each msgset's raw bytes back-to-back */
+ for (i = 0; i < rd_list_cnt(&msgsets); i++) {
+ const rd_kafka_mock_msgset_t *mset = rd_list_elem(&msgsets, i);
+ rd_kafka_buf_write(resp, mset->bytes.data,
+ RD_KAFKAP_BYTES_LEN(&mset->bytes));
+ }
+
+ rd_list_destroy(&msgsets);
+ return total_len;
+}
+
+static int rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn,
+ rd_kafka_buf_t *rkbuf) {
+ const rd_bool_t log_decode_errors = rd_true;
+ rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster;
+ rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf);
+ rd_kafkap_str_t GroupId, MemberId;
+ int32_t SessionEpoch = -1, MaxWaitMs = 0, MinBytes = 0, MaxBytes = 0,
+ MaxRecords = 0, BatchSize = 0;
+ int32_t TopicsCnt;
+ rd_kafka_topic_partition_list_t *requested_partitions = NULL;
+ rd_kafka_topic_partition_list_t *forgotten_partitions = NULL;
+ rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
+ rd_kafka_mock_sharegroup_t *sgrp = NULL;
+ rd_kafka_mock_sgrp_fetch_session_t *session = NULL;
+
+ (void)log_decode_errors;
+
+ rd_kafka_buf_read_str(rkbuf, &GroupId);
+ rd_kafka_buf_read_str(rkbuf, &MemberId);
+ /* KIP-932: ShareFetch has ShareSessionEpoch only, no SessionId.
+ * Sessions are keyed by (GroupId, MemberId). */
+ rd_kafka_buf_read_i32(rkbuf, &SessionEpoch);
+ rd_kafka_buf_read_i32(rkbuf, &MaxWaitMs);
+ rd_kafka_buf_read_i32(rkbuf, &MinBytes);
+ rd_kafka_buf_read_i32(rkbuf, &MaxBytes);
+ rd_kafka_buf_read_i32(rkbuf, &MaxRecords);
+ rd_kafka_buf_read_i32(rkbuf, &BatchSize);
+
+ requested_partitions = rd_kafka_topic_partition_list_new(0);
+
+ rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, RD_KAFKAP_TOPICS_MAX);
+ while (TopicsCnt-- > 0) {
+ rd_kafka_Uuid_t TopicId = RD_KAFKA_UUID_ZERO;
+ int32_t PartitionCnt;
+
+ rd_kafka_buf_read_uuid(rkbuf, &TopicId);
+ rd_kafka_buf_read_arraycnt(rkbuf, &PartitionCnt,
+ RD_KAFKAP_PARTITIONS_MAX);
+
+ while (PartitionCnt-- > 0) {
+ int32_t Partition;
+ int32_t AckBatchCnt;
+ rd_kafka_topic_partition_t *rktpar;
+
+ rd_kafka_buf_read_i32(rkbuf, &Partition);
+ rd_kafka_buf_read_arraycnt(rkbuf, &AckBatchCnt, -1);
+ while (AckBatchCnt-- > 0) {
+ int32_t AckTypeCnt;
+ int8_t AckType;
+ rd_kafka_buf_read_i64(rkbuf,
+ NULL); /* FirstOffset */
+ rd_kafka_buf_read_i64(rkbuf,
+ NULL); /* LastOffset */
+ rd_kafka_buf_read_arraycnt(rkbuf, &AckTypeCnt,
+ -1);
+ while (AckTypeCnt-- > 0) {
+ rd_kafka_buf_read_i8(rkbuf, &AckType);
+ (void)AckType;
+ }
+ rd_kafka_buf_skip_tags(rkbuf);
+ }
+
+ rktpar = rd_kafka_topic_partition_list_add(
+ requested_partitions, "", Partition);
+ rd_kafka_topic_partition_set_topic_id(rktpar, TopicId);
+
+ rd_kafka_buf_skip_tags(rkbuf);
+ }
+
+ rd_kafka_buf_skip_tags(rkbuf);
+ }
+
+ /* ForgottenTopicsData */
+ {
+ int32_t ForgottenTopicsCnt;
+ rd_kafka_buf_read_arraycnt(rkbuf, &ForgottenTopicsCnt,
+ RD_KAFKAP_TOPICS_MAX);
+ if (ForgottenTopicsCnt > 0)
+ forgotten_partitions =
+ rd_kafka_topic_partition_list_new(
+ ForgottenTopicsCnt);
+ while (ForgottenTopicsCnt-- > 0) {
+ rd_kafka_Uuid_t ForgTopicId = RD_KAFKA_UUID_ZERO;
+ int32_t ForgPartCnt;
+ rd_kafka_buf_read_uuid(rkbuf, &ForgTopicId);
+ rd_kafka_buf_read_arraycnt(rkbuf, &ForgPartCnt,
+ RD_KAFKAP_PARTITIONS_MAX);
+ while (ForgPartCnt-- > 0) {
+ int32_t ForgPartition;
+ rd_kafka_topic_partition_t *ftp;
+ rd_kafka_buf_read_i32(rkbuf, &ForgPartition);
+
+ /* Record in forgotten list for session
+ * removal and inflight release. */
+ ftp = rd_kafka_topic_partition_list_add(
+ forgotten_partitions, "", ForgPartition);
+ rd_kafka_topic_partition_set_topic_id(
+ ftp, ForgTopicId);
+
+ /* Remove from requested_partitions so they
+ * are not fetched in this request. */
+ if (requested_partitions) {
+ int idx =
+ rd_kafka_topic_partition_list_find_idx_by_id(
+ requested_partitions,
+ ForgTopicId, ForgPartition);
+ if (idx >= 0)
+ rd_kafka_topic_partition_list_del_by_idx(
+ requested_partitions, idx);
+ }
+ }
+ /* ForgottenTopic tags */
+ rd_kafka_buf_skip_tags(rkbuf);
+ }
+ }
+
+ /* Top-level tags */
+ rd_kafka_buf_skip_tags(rkbuf);
+
+ rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK",
+ "ShareFetch parsed: group %.*s member %.*s "
+ "session_epoch %" PRId32 " max_wait %" PRId32
+ " min_bytes %" PRId32 " max_bytes %" PRId32
+ " max_records %" PRId32 " batch_size %" PRId32,
+ RD_KAFKAP_STR_PR(&GroupId), RD_KAFKAP_STR_PR(&MemberId),
+ SessionEpoch, MaxWaitMs, MinBytes, MaxBytes, MaxRecords,
+ BatchSize);
+
+ err = rd_kafka_mock_next_request_error(mconn, resp);
+
+ if (!err) {
+ int64_t remaining_records =
+ MaxRecords > 0 ? (int64_t)MaxRecords : -1;
+ int64_t remaining_bytes = MaxBytes > 0 ? (int64_t)MaxBytes : -1;
+ int acquired_cnt = 0;
+ int64_t acquired_bytes = 0;
+ rd_ts_t now = rd_clock();
+
+ /* KIP-932 session management.
+ * Sessions are keyed by (GroupId, MemberId).
+ * SessionEpoch: 0 = open new session,
+ * -1 = close session,
+ * >0 = continue (must match expected epoch). */
+ mtx_lock(&mcluster->lock);
+ sgrp = rd_kafka_mock_sharegroup_get(mcluster, &GroupId);
+
+ /* Member validation: verify the MemberId is a registered
+ * member of this share group (joined via SGHB). */
+ if (!rd_kafka_mock_sharegroup_member_find(sgrp, &MemberId)) {
+ err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID;
+ rd_kafka_dbg(
+ mconn->broker->cluster->rk, MOCK, "MOCK",
+ "ShareFetch: unknown member %.*s in group %.*s",
+ RD_KAFKAP_STR_PR(&MemberId),
+ RD_KAFKAP_STR_PR(&GroupId));
+ }
+
+ /* Look up existing session by MemberId */
+ if (!err) {
+ TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) {
+ if (!rd_kafkap_str_cmp_str(&MemberId,
+ session->member_id))
+ break;
+ }
+ }
+
+ if (!err && SessionEpoch == 0) {
+ /* Open a new session (or reuse if one already exists
+ * for this member). */
+ if (!session) {
+ session = rd_calloc(1, sizeof(*session));
+ session->member_id =
+ RD_KAFKAP_STR_DUP(&MemberId);
+ session->session_epoch = 0;
+ session->partitions =
+ rd_kafka_topic_partition_list_copy(
+ requested_partitions);
+ TAILQ_INSERT_TAIL(&sgrp->fetch_sessions,
+ session, link);
+ sgrp->fetch_session_cnt++;
+ } else {
+ /* Session already exists for this member;
+ * reset it. */
+ session->session_epoch = 0;
+ RD_IF_FREE(
+ session->partitions,
+ rd_kafka_topic_partition_list_destroy);
+ session->partitions =
+ rd_kafka_topic_partition_list_copy(
+ requested_partitions);
+ }
+ } else if (!err && SessionEpoch == -1) {
+ /* Close the session and release all locks held
+ * by this member. */
+ if (session) {
+ rd_kafka_mock_sgrp_release_member_locks(
+ sgrp, session->member_id);
+ TAILQ_REMOVE(&sgrp->fetch_sessions, session,
+ link);
+ sgrp->fetch_session_cnt--;
+ rd_kafka_mock_sgrp_fetch_session_destroy(
+ session);
+ session = NULL;
+ }
+ /* Closing a non-existent session is not an error;
+ * proceed with an empty response. */
+ } else if (!err) {
+ /* Continue existing session: validate epoch. */
+ if (!session) {
+ err =
+ RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH;
+ } else if (SessionEpoch != session->session_epoch) {
+ err =
+ RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH;
+ } else {
+ /* Update partition list if changed */
+ if (!rd_kafka_mock_tplist_equal_by_id(
+ requested_partitions,
+ session->partitions)) {
+ RD_IF_FREE(
+ session->partitions,
+ rd_kafka_topic_partition_list_destroy);
+ session->partitions =
+ rd_kafka_topic_partition_list_copy(
+ requested_partitions);
+ }
+ }
+ }
+
+ /* For all successful, non-close requests: update activity
+ * timestamp and increment epoch for next request. */
+ if (!err && session) {
+ session->ts_last_activity = rd_clock();
+ session->session_epoch++;
+ }
+
+ /* Remove forgotten partitions from session and release
+ * any in-flight ACQUIRED records owned by this member. */
+ if (!err && session && forgotten_partitions &&
+ forgotten_partitions->cnt > 0) {
+ rd_kafka_topic_partition_t *ftp;
+ RD_KAFKA_TPLIST_FOREACH(ftp, forgotten_partitions) {
+ rd_kafka_Uuid_t ftid =
+ rd_kafka_topic_partition_get_topic_id(ftp);
+ rd_kafka_mock_sgrp_partmeta_t *pmeta;
+
+ /* Remove from session partition list */
+ if (session->partitions) {
+ int idx =
+ rd_kafka_topic_partition_list_find_idx_by_id(
+ session->partitions, ftid,
+ ftp->partition);
+ if (idx >= 0)
+ rd_kafka_topic_partition_list_del_by_idx(
+ session->partitions, idx);
+ }
+
+ /* Release ACQUIRED records for this
+ * member on this partition. */
+ pmeta = rd_kafka_mock_sgrp_partmeta_find(
+ sgrp, ftid, ftp->partition);
+ if (pmeta) {
+ rd_kafka_mock_sgrp_record_state_t
+ *state,
+ *tmp;
+ TAILQ_FOREACH_SAFE(state,
+ &pmeta->inflight,
+ link, tmp) {
+ if (state->state !=
+ RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED)
+ continue;
+ if (!state->owner_member_id)
+ continue;
+ if (rd_kafkap_str_cmp_str(
+ &MemberId,
+ state->owner_member_id))
+ continue;
+ /* Release: mark AVAILABLE */
+ state->state =
+ RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE;
+ rd_free(state->owner_member_id);
+ state->owner_member_id = NULL;
+ state->lock_expiry_ts = 0;
+ }
+ }
+ }
+ }
+
+ if (!err && sgrp) {
+ rd_kafka_topic_partition_t *rktpar;
+ RD_KAFKA_TPLIST_FOREACH(rktpar, requested_partitions) {
+ rd_kafka_Uuid_t topic_id =
+ rd_kafka_topic_partition_get_topic_id(
+ rktpar);
+ rd_kafka_mock_topic_t *mtopic =
+ rd_kafka_mock_topic_find_by_id(mcluster,
+ topic_id);
+ rd_kafka_mock_partition_t *mpart;
+
+ if (!mtopic) {
+ err =
+ RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART;
+ break;
+ }
+
+ mpart = rd_kafka_mock_partition_find(
+ mtopic, rktpar->partition);
+ if (!mpart) {
+ err =
+ RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART;
+ break;
+ }
+
+ rd_kafka_mock_sgrp_partmeta_t *pmeta =
+ rd_kafka_mock_sgrp_partmeta_get(
+ sgrp, topic_id, rktpar->partition,
+ mpart);
+ rd_kafka_mock_sgrp_partmeta_prune_archived(
+ pmeta);
+ rd_kafka_mock_sgrp_acquire_available_offsets(
+ pmeta, mpart, &MemberId,
+ now + ((sgrp->record_lock_duration_ms > 0
+ ? sgrp->record_lock_duration_ms
+ : sgrp->session_timeout_ms) *
+ 1000),
+ sgrp->max_delivery_attempts,
+ MaxRecords > 0 ? &remaining_records : NULL,
+ MaxBytes > 0 ? &remaining_bytes : NULL,
+ &acquired_cnt, &acquired_bytes);
+ }
+ }
+
+ rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK",
+ "ShareFetch acquired: %d records, %" PRId64
+ " bytes",
+ acquired_cnt, acquired_bytes);
+
+ /* Response: ThrottleTimeMs */
+ rd_kafka_buf_write_i32(resp, 0);
+ /* Response: ErrorCode */
+ rd_kafka_buf_write_i16(resp, err);
+ /* Response: ErrorMessage */
+ if (err)
+ rd_kafka_buf_write_str(resp, rd_kafka_err2str(err), -1);
+ else
+ rd_kafka_buf_write_str(resp, NULL, -1);
+ /* Response: AcquisitionLockTimeoutMs */
+ rd_kafka_buf_write_i32(resp,
+ sgrp ? sgrp->session_timeout_ms : 0);
+
+ rd_kafka_topic_partition_list_sort_by_topic_id(
+ requested_partitions);
+
+ {
+ int i = 0;
+ int topic_cnt = 0;
+ rd_kafka_Uuid_t current_topic = RD_KAFKA_UUID_ZERO;
+
+ for (i = 0; i < requested_partitions->cnt; i++) {
+ rd_kafka_topic_partition_t *rktpar =
+ &requested_partitions->elems[i];
+ rd_kafka_Uuid_t topic_id =
+ rd_kafka_topic_partition_get_topic_id(
+ rktpar);
+ if (i == 0 ||
+ rd_kafka_Uuid_cmp(topic_id,
+ current_topic) != 0) {
+ topic_cnt++;
+ current_topic = topic_id;
+ }
+ }
+
+ /* Response: #Topics */
+ rd_kafka_buf_write_arraycnt(resp, topic_cnt);
+
+ i = 0;
+ while (i < requested_partitions->cnt) {
+ int j;
+ rd_kafka_Uuid_t topic_id =
+ rd_kafka_topic_partition_get_topic_id(
+ &requested_partitions->elems[i]);
+ int part_cnt = 0;
+
+ for (j = i; j < requested_partitions->cnt;
+ j++) {
+ rd_kafka_Uuid_t next_topic_id =
+ rd_kafka_topic_partition_get_topic_id(
+ &requested_partitions
+ ->elems[j]);
+ if (rd_kafka_Uuid_cmp(
+ topic_id, next_topic_id) != 0)
+ break;
+ part_cnt++;
+ }
+
+ /* Response: TopicId */
+ rd_kafka_buf_write_uuid(resp, &topic_id);
+ /* Response: #Partitions */
+ rd_kafka_buf_write_arraycnt(resp, part_cnt);
+
+ for (j = i; j < i + part_cnt; j++) {
+ rd_kafka_topic_partition_t *rktpar =
+ &requested_partitions->elems[j];
+ rd_kafka_mock_topic_t *mtopic =
+ rd_kafka_mock_topic_find_by_id(
+ mcluster, topic_id);
+ rd_kafka_mock_partition_t *mpart =
+ mtopic
+ ? rd_kafka_mock_partition_find(
+ mtopic, rktpar->partition)
+ : NULL;
+ rd_kafka_mock_sgrp_partmeta_t *pmeta =
+ mpart
+ ? rd_kafka_mock_sgrp_partmeta_find(
+ sgrp, topic_id,
+ rktpar->partition)
+ : NULL;
+ rd_kafka_resp_err_t part_err =
+ mpart
+ ? RD_KAFKA_RESP_ERR_NO_ERROR
+ : RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART;
+
+ /* Response: Partition */
+ rd_kafka_buf_write_i32(
+ resp, rktpar->partition);
+ /* Response: PartitionFetchErrorCode */
+ rd_kafka_buf_write_i16(resp, part_err);
+ /* Response: PartitionFetchErrorString
+ */
+ if (part_err)
+ rd_kafka_buf_write_str(
+ resp,
+ rd_kafka_err2str(part_err),
+ -1);
+ else
+ rd_kafka_buf_write_str(
+ resp, NULL, -1);
+ /* Response: AcknowledgementErrorCode */
+ rd_kafka_buf_write_i16(resp, 0);
+ /* Response: AcknowledgementErrorString
+ */
+ rd_kafka_buf_write_str(resp, NULL, -1);
+ /* Response: CurrentLeader */
+ rd_kafka_buf_write_i32(resp, -1);
+ rd_kafka_buf_write_i32(resp, -1);
+ rd_kafka_buf_write_tags_empty(resp);
+ /* Response: Records (all acquired
+ * batches concatenated) */
+ if (mpart && pmeta)
+ rd_kafka_mock_sgrp_write_acquired_records(
+ resp, pmeta, mpart,
+ &MemberId, now);
+ else
+ rd_kafka_buf_write_kbytes(resp,
+ NULL);
+ /* Response: AcquiredRecords */
+ rd_kafka_buf_write_arraycnt(resp, 0);
+ /* Response: Partition tags */
+ rd_kafka_buf_write_tags_empty(resp);
+ }
+
+ /* Response: Topic tags */
+ rd_kafka_buf_write_tags_empty(resp);
+
+ i += part_cnt;
+ }
+ }
+
+ /* Response: NodeEndpoints */
+ rd_kafka_buf_write_arraycnt(resp, 0);
+ /* Response: Top-level tags */
+ rd_kafka_buf_write_tags_empty(resp);
+
+ mtx_unlock(&mcluster->lock);
+
+ rd_kafka_mock_connection_send_response0(mconn, resp, rd_true);
+
+ rd_kafka_topic_partition_list_destroy(requested_partitions);
+ RD_IF_FREE(forgotten_partitions,
+ rd_kafka_topic_partition_list_destroy);
+ return 0;
+ }
+
+ /* Error response */
+ rd_kafka_buf_write_i32(resp, 0);
+ rd_kafka_buf_write_i16(resp, err);
+ rd_kafka_buf_write_str(resp, rd_kafka_err2str(err), -1);
+ rd_kafka_buf_write_i32(resp, 0);
+ rd_kafka_buf_write_arraycnt(resp, 0);
+ rd_kafka_buf_write_arraycnt(resp, 0);
+ rd_kafka_buf_write_tags_empty(resp);
+
+ rd_kafka_mock_connection_send_response0(mconn, resp, rd_true);
+ rd_kafka_topic_partition_list_destroy(requested_partitions);
+ RD_IF_FREE(forgotten_partitions, rd_kafka_topic_partition_list_destroy);
+ return 0;
+
+err_parse:
+ RD_IF_FREE(requested_partitions, rd_kafka_topic_partition_list_destroy);
+ RD_IF_FREE(forgotten_partitions, rd_kafka_topic_partition_list_destroy);
+ rd_kafka_buf_destroy(resp);
+ return -1;
+}
+
/**
* @brief Default request handlers
*/
@@ -3025,10 +4041,13 @@ const struct rd_kafka_mock_api_handler
{2, 2, -1, rd_kafka_mock_handle_OffsetForLeaderEpoch},
[RD_KAFKAP_ConsumerGroupHeartbeat] =
{1, 1, 1, rd_kafka_mock_handle_ConsumerGroupHeartbeat},
+ [RD_KAFKAP_ShareGroupHeartbeat] =
+ {1, 1, 1, rd_kafka_mock_handle_ShareGroupHeartbeat},
[RD_KAFKAP_GetTelemetrySubscriptions] =
{0, 0, 0, rd_kafka_mock_handle_GetTelemetrySubscriptions},
[RD_KAFKAP_PushTelemetry] = {0, 0, 0,
rd_kafka_mock_handle_PushTelemetry},
+ [RD_KAFKAP_ShareFetch] = {1, 1, 1, rd_kafka_mock_handle_ShareFetch},
};
diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h
index 2ef7a2a339..31ebb41421 100644
--- a/src/rdkafka_mock_int.h
+++ b/src/rdkafka_mock_int.h
@@ -179,6 +179,100 @@ typedef struct rd_kafka_mock_cgrp_consumer_member_s {
rd_kafka_mock_cgrp_consumer_t *mcgrp; /**< Consumer group */
} rd_kafka_mock_cgrp_consumer_member_t;
+/**
+ * @brief Share record state.
+ */
+enum rd_kafka_mock_sgrp_record_state_e {
+ RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE = 0,
+ RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED = 1,
+ RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED = 2
+};
+
+typedef struct rd_kafka_mock_sgrp_record_state_s {
+ TAILQ_ENTRY(rd_kafka_mock_sgrp_record_state_s) link;
+ int64_t offset;
+ char *owner_member_id;
+ rd_ts_t lock_expiry_ts;
+ int32_t delivery_count;
+ enum rd_kafka_mock_sgrp_record_state_e state;
+} rd_kafka_mock_sgrp_record_state_t;
+
+/**
+ * @brief Share partition metadata.
+ */
+typedef struct rd_kafka_mock_sgrp_partmeta_s {
+ TAILQ_ENTRY(rd_kafka_mock_sgrp_partmeta_s) link;
+ rd_kafka_Uuid_t topic_id;
+ int32_t partition;
+ int64_t spso;
+ int64_t speo;
+ TAILQ_HEAD(, rd_kafka_mock_sgrp_record_state_s) inflight;
+ int inflight_cnt;
+} rd_kafka_mock_sgrp_partmeta_t;
+
+/**
+ * @brief Share fetch session.
+ */
+typedef struct rd_kafka_mock_sgrp_fetch_session_s {
+ TAILQ_ENTRY(rd_kafka_mock_sgrp_fetch_session_s) link;
+ char *member_id;
+ int32_t session_id;
+ int32_t session_epoch;
+ rd_ts_t ts_last_activity;
+ rd_kafka_topic_partition_list_t *partitions;
+} rd_kafka_mock_sgrp_fetch_session_t;
+
+/**
+ * @struct Share group (KIP-932).
+ */
+typedef struct rd_kafka_mock_sharegroup_s {
+ TAILQ_ENTRY(rd_kafka_mock_sharegroup_s) link;
+ struct rd_kafka_mock_cluster_s *cluster; /**< Cluster */
+ char *id; /**< Share group Id */
+ int32_t group_epoch; /**< Group epoch */
+ int session_timeout_ms; /**< Session timeout */
+ rd_kafka_timer_t session_tmr; /**< Session timeout timer */
+ int heartbeat_interval_ms; /**< Heartbeat interval */
+ TAILQ_HEAD(, rd_kafka_mock_sharegroup_member_s)
+ members; /**< Share group members */
+ int member_cnt; /**< Number of share group members */
+ rd_bool_t manual_assignment; /**< Use manual assignment */
+
+ /* ShareFetch state (KIP-932) */
+ TAILQ_HEAD(, rd_kafka_mock_sgrp_partmeta_s)
+ partitions; /**< Share partition metadata */
+ int partition_cnt; /**< Number of partitions */
+ TAILQ_HEAD(, rd_kafka_mock_sgrp_fetch_session_s)
+ fetch_sessions; /**< Active fetch sessions */
+ int fetch_session_cnt; /**< Number of fetch sessions */
+ rd_kafka_timer_t fetch_session_tmr; /**< Fetch session expiry timer */
+
+ /* Per-record limits */
+ int max_delivery_attempts; /**< Max times a record can be
+ * acquired before being archived.
+ * 0 = unlimited (default 5). */
+ int record_lock_duration_ms; /**< Per-record lock duration in ms.
+ * 0 = use session_timeout_ms
+ * as fallback (default 0). */
+} rd_kafka_mock_sharegroup_t;
+
+/**
+ * @struct Share group member (KIP-932).
+ */
+typedef struct rd_kafka_mock_sharegroup_member_s {
+ TAILQ_ENTRY(rd_kafka_mock_sharegroup_member_s) link;
+ char *id; /**< MemberId */
+ rd_ts_t ts_last_activity; /**< Last heartbeat timestamp */
+ int32_t member_epoch; /**< Current member epoch */
+ int32_t previous_member_epoch; /**< Previous member epoch (allows
+ * client to catch up if response
+ * with bumped epoch was lost) */
+ rd_list_t *subscribed_topic_names; /**< Subscribed topic names */
+ rd_kafka_topic_partition_list_t *assignment; /**< Current assignment */
+ struct rd_kafka_mock_connection_s *conn; /**< Connection */
+ rd_kafka_mock_sharegroup_t *mshgrp; /**< Share group */
+} rd_kafka_mock_sharegroup_member_t;
+
/**
* @struct TransactionalId + PID (+ optional sequence state)
@@ -429,6 +523,8 @@ struct rd_kafka_mock_cluster_s {
TAILQ_HEAD(, rd_kafka_mock_cgrp_consumer_s) cgrps_consumer;
+ TAILQ_HEAD(, rd_kafka_mock_sharegroup_s) sharegrps;
+
/** Explicit coordinators (set with mock_set_coordinator()) */
TAILQ_HEAD(, rd_kafka_mock_coord_s) coords;
@@ -467,6 +563,16 @@ struct rd_kafka_mock_cluster_s {
int group_consumer_session_timeout_ms;
/** Heartbeat interval (KIP 848) */
int group_consumer_heartbeat_interval_ms;
+ /** Session timeout (KIP 932) */
+ int sharegroup_session_timeout_ms;
+ /** Heartbeat interval (KIP 932) */
+ int sharegroup_heartbeat_interval_ms;
+ /** Max delivery attempts per record (KIP 932).
+ * 0 = unlimited. */
+ int sharegroup_max_delivery_attempts;
+ /** Per-record lock duration in ms (KIP 932).
+ * 0 = use session_timeout_ms. */
+ int sharegroup_record_lock_duration_ms;
} defaults;
/**< Dynamic array of IO handlers for corresponding fd in .fds */
@@ -541,6 +647,13 @@ rd_kafka_mock_topic_t *
rd_kafka_mock_topic_find_by_id(const rd_kafka_mock_cluster_t *mcluster,
rd_kafka_Uuid_t id);
+void rd_kafka_mock_sgrp_fetch_session_destroy(
+ rd_kafka_mock_sgrp_fetch_session_t *session);
+void rd_kafka_mock_sgrp_release_member_locks(rd_kafka_mock_sharegroup_t *mshgrp,
+ const char *member_id);
+void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts,
+ void *arg);
+
rd_kafka_mock_broker_t *
rd_kafka_mock_cluster_get_coord(rd_kafka_mock_cluster_t *mcluster,
rd_kafka_coordtype_t KeyType,
@@ -700,6 +813,51 @@ rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_add(
void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster,
rd_kafka_mock_connection_t *mconn);
+
+/* Share group (KIP-932) */
+
+void rd_kafka_mock_sharegrps_init(rd_kafka_mock_cluster_t *mcluster);
+
+rd_kafka_mock_sharegroup_t *
+rd_kafka_mock_sharegroup_find(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId);
+
+rd_kafka_mock_sharegroup_t *
+rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId);
+
+void rd_kafka_mock_sharegroup_destroy(rd_kafka_mock_sharegroup_t *mshgrp);
+
+rd_kafka_mock_sharegroup_member_t *
+rd_kafka_mock_sharegroup_member_find(rd_kafka_mock_sharegroup_t *mshgrp,
+ const rd_kafkap_str_t *MemberId);
+
+void rd_kafka_mock_sharegroup_member_destroy(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_kafka_mock_sharegroup_member_t *member);
+
+void rd_kafka_mock_sharegroup_member_active(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_kafka_mock_sharegroup_member_t *member);
+
+rd_kafka_mock_sharegroup_member_t *
+rd_kafka_mock_sharegroup_member_get(rd_kafka_mock_sharegroup_t *mshgrp,
+ const rd_kafkap_str_t *MemberID,
+ int32_t MemberEpoch,
+ rd_kafka_mock_connection_t *mconn);
+
+rd_bool_t rd_kafka_mock_sharegroup_member_subscribed_topic_names_set(
+ rd_kafka_mock_sharegroup_member_t *member,
+ const rd_kafkap_str_t *SubscribedTopicNames,
+ int32_t SubscribedTopicNamesCnt);
+
+void rd_kafka_mock_sharegroup_assignment_recalculate(
+ rd_kafka_mock_sharegroup_t *mshgrp);
+
+void rd_kafka_mock_sharegrps_connection_closed(
+ rd_kafka_mock_cluster_t *mcluster,
+ rd_kafka_mock_connection_t *mconn);
+
/**
*@}
*/
diff --git a/src/rdkafka_mock_sharegrp.c b/src/rdkafka_mock_sharegrp.c
new file mode 100644
index 0000000000..38589a298e
--- /dev/null
+++ b/src/rdkafka_mock_sharegrp.c
@@ -0,0 +1,931 @@
+/*
+ * 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.
+ */
+
+/**
+ * Mocks
+ *
+ */
+
+
+#include "rdkafka_int.h"
+#include "rdbuf.h"
+#include "rdkafka_mock_int.h"
+#include "rdkafka_mock_group_common.h"
+
+/**
+ * @brief Share group target assignment (manual)
+ */
+typedef struct rd_kafka_mock_sharegroup_target_assignments_s {
+ rd_list_t member_ids; /**< List of member ids (char *) */
+ rd_list_t assignment; /**< List of rd_kafka_topic_partition_list_t */
+} rd_kafka_mock_sharegroup_target_assignment_t;
+
+/* Forward declarations */
+static void rd_kafka_mock_sharegroup_session_tmr_cb(rd_kafka_timers_t *rkts,
+ void *arg);
+
+/**
+ * @brief Initializes sharegroups in mock cluster
+ */
+void rd_kafka_mock_sharegrps_init(rd_kafka_mock_cluster_t *mcluster) {
+ TAILQ_INIT(&mcluster->sharegrps);
+ mcluster->defaults.sharegroup_session_timeout_ms = 45000;
+ mcluster->defaults.sharegroup_heartbeat_interval_ms = 5000;
+ mcluster->defaults.sharegroup_max_delivery_attempts = 5;
+ mcluster->defaults.sharegroup_record_lock_duration_ms = 0;
+}
+
+/**
+ * @brief Find a share group by GroupId.
+ */
+rd_kafka_mock_sharegroup_t *
+rd_kafka_mock_sharegroup_find(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupId) {
+ return RD_KAFKA_MOCK_GROUP_FIND(&mcluster->sharegrps, GroupId,
+ rd_kafka_mock_sharegroup_t);
+}
+
+/**
+ * @brief Get or create a share group
+ */
+rd_kafka_mock_sharegroup_t *
+rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster,
+ const rd_kafkap_str_t *GroupID) {
+ rd_kafka_mock_sharegroup_t *mshgrp;
+
+ /* Check if the share group already exists */
+ mshgrp = rd_kafka_mock_sharegroup_find(mcluster, GroupID);
+ if (mshgrp)
+ return mshgrp;
+
+ /* Create new share group */
+ mshgrp = rd_calloc(1, sizeof(*mshgrp));
+ mshgrp->cluster = mcluster;
+ mshgrp->id = RD_KAFKAP_STR_DUP(GroupID);
+ mshgrp->group_epoch = 1;
+ mshgrp->session_timeout_ms =
+ mcluster->defaults.sharegroup_session_timeout_ms;
+ mshgrp->heartbeat_interval_ms =
+ mcluster->defaults.sharegroup_heartbeat_interval_ms;
+
+ TAILQ_INIT(&mshgrp->members);
+ mshgrp->member_cnt = 0;
+
+ /* ShareFetch state */
+ TAILQ_INIT(&mshgrp->partitions);
+ TAILQ_INIT(&mshgrp->fetch_sessions);
+ mshgrp->partition_cnt = 0;
+ mshgrp->fetch_session_cnt = 0;
+
+ /* Per-record limits */
+ mshgrp->max_delivery_attempts =
+ mcluster->defaults.sharegroup_max_delivery_attempts;
+ mshgrp->record_lock_duration_ms =
+ mcluster->defaults.sharegroup_record_lock_duration_ms;
+
+ rd_kafka_timer_start(&mcluster->timers, &mshgrp->session_tmr,
+ 1000 * 1000 /* 1s */,
+ rd_kafka_mock_sharegroup_session_tmr_cb, mshgrp);
+
+ /* Fetch session expiry timer */
+ rd_kafka_timer_start(&mcluster->timers, &mshgrp->fetch_session_tmr,
+ 1000 * 1000 /* 1s */,
+ rd_kafka_mock_sgrp_fetch_session_tmr_cb, mshgrp);
+
+ TAILQ_INSERT_TAIL(&mcluster->sharegrps, mshgrp, link);
+
+ return mshgrp;
+}
+
+/**
+ * @brief Destroy a share group
+ */
+void rd_kafka_mock_sharegroup_destroy(rd_kafka_mock_sharegroup_t *mshgrp) {
+ rd_kafka_mock_sharegroup_member_t *member;
+ rd_kafka_mock_sgrp_partmeta_t *pmeta;
+ rd_kafka_mock_sgrp_fetch_session_t *session;
+
+ TAILQ_REMOVE(&mshgrp->cluster->sharegrps, mshgrp, link);
+ rd_kafka_timer_stop(&mshgrp->cluster->timers, &mshgrp->session_tmr,
+ RD_DO_LOCK);
+ rd_kafka_timer_stop(&mshgrp->cluster->timers,
+ &mshgrp->fetch_session_tmr, RD_DO_LOCK);
+
+ /* Destroy all members */
+ while ((member = TAILQ_FIRST(&mshgrp->members)))
+ rd_kafka_mock_sharegroup_member_destroy(mshgrp, member);
+
+ /* Destroy ShareFetch partition metadata */
+ while ((pmeta = TAILQ_FIRST(&mshgrp->partitions))) {
+ rd_kafka_mock_sgrp_record_state_t *state;
+ TAILQ_REMOVE(&mshgrp->partitions, pmeta, link);
+ while ((state = TAILQ_FIRST(&pmeta->inflight))) {
+ TAILQ_REMOVE(&pmeta->inflight, state, link);
+ rd_free(state->owner_member_id);
+ rd_free(state);
+ }
+ rd_free(pmeta);
+ }
+
+ /* Destroy ShareFetch sessions */
+ while ((session = TAILQ_FIRST(&mshgrp->fetch_sessions))) {
+ TAILQ_REMOVE(&mshgrp->fetch_sessions, session, link);
+ mshgrp->fetch_session_cnt--;
+ rd_kafka_mock_sgrp_fetch_session_destroy(session);
+ }
+
+ rd_free(mshgrp->id);
+ rd_free(mshgrp);
+}
+
+/**
+ * @brief Find a share group member by MemberId.
+ */
+rd_kafka_mock_sharegroup_member_t *
+rd_kafka_mock_sharegroup_member_find(rd_kafka_mock_sharegroup_t *mshgrp,
+ const rd_kafkap_str_t *MemberId) {
+ return RD_KAFKA_MOCK_MEMBER_FIND(&mshgrp->members, MemberId,
+ rd_kafka_mock_sharegroup_member_t);
+}
+
+/**
+ * @brief Destroy a share group member.
+ */
+void rd_kafka_mock_sharegroup_member_destroy(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_kafka_mock_sharegroup_member_t *member) {
+ rd_assert(mshgrp->member_cnt > 0);
+ TAILQ_REMOVE(&mshgrp->members, member, link);
+ mshgrp->member_cnt--;
+ rd_free(member->id);
+
+ RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy_free);
+ RD_IF_FREE(member->assignment, rd_kafka_topic_partition_list_destroy);
+ rd_free(member);
+}
+
+/**
+ * @brief Mark member as active.
+ */
+void rd_kafka_mock_sharegroup_member_active(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_kafka_mock_sharegroup_member_t *member) {
+ rd_kafka_mock_group_member_mark_active(mshgrp->cluster->rk, "share",
+ member->id,
+ &member->ts_last_activity);
+}
+
+/**
+ * @brief Fence a member.
+ */
+void rd_kafka_mock_sharegroup_member_fenced(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_kafka_mock_sharegroup_member_t *member) {
+ rd_kafka_dbg(mshgrp->cluster->rk, MOCK, "MOCK",
+ "Member %s is fenced from sharegroup %s", member->id,
+ mshgrp->id);
+
+ rd_kafka_mock_sharegroup_member_destroy(mshgrp, member);
+
+ /* Recalculate assignments so remaining members get the
+ * freed partitions. */
+ rd_kafka_mock_sharegroup_assignment_recalculate(mshgrp);
+}
+
+/**
+ * @brief Check all members for inactivity and remove them if timed out.
+ */
+static void rd_kafka_mock_sharegroup_session_tmr_cb(rd_kafka_timers_t *rkts,
+ void *arg) {
+ rd_kafka_mock_sharegroup_t *mshgrp = arg;
+ rd_kafka_mock_sharegroup_member_t *member, *tmp;
+ rd_ts_t now = rd_clock();
+ rd_kafka_mock_cluster_t *mcluster = mshgrp->cluster;
+
+ mtx_lock(&mcluster->lock);
+ TAILQ_FOREACH_SAFE(member, &mshgrp->members, link, tmp) {
+ if (member->ts_last_activity +
+ (mshgrp->session_timeout_ms * 1000) >
+ now)
+ continue;
+
+ rd_kafka_dbg(mcluster->rk, MOCK, "MOCK",
+ "Member %s session timed out for sharegroup %s",
+ member->id, mshgrp->id);
+
+ rd_kafka_mock_sharegroup_member_fenced(mshgrp, member);
+ }
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief Get or create a share group member.
+ */
+rd_kafka_mock_sharegroup_member_t *
+rd_kafka_mock_sharegroup_member_get(rd_kafka_mock_sharegroup_t *mshgrp,
+ const rd_kafkap_str_t *MemberID,
+ int32_t MemberEpoch,
+ rd_kafka_mock_connection_t *mconn) {
+ rd_kafka_mock_sharegroup_member_t *member;
+
+ /* Check if the member already exists */
+ member = rd_kafka_mock_sharegroup_member_find(mshgrp, MemberID);
+ if (member) {
+ member->conn = mconn;
+ rd_kafka_mock_sharegroup_member_active(mshgrp, member);
+ return member;
+ }
+
+ /* Only create if epoch is 0 */
+ if (MemberEpoch != 0)
+ return NULL;
+
+ /* Create new member */
+ member = rd_calloc(1, sizeof(*member));
+ member->mshgrp = mshgrp;
+ member->id = RD_KAFKAP_STR_DUP(MemberID);
+ member->member_epoch = mshgrp->group_epoch;
+ member->previous_member_epoch =
+ -1; /* No previous epoch for new members */
+ member->conn = mconn;
+
+ TAILQ_INSERT_TAIL(&mshgrp->members, member, link);
+ mshgrp->member_cnt++;
+ rd_kafka_mock_sharegroup_member_active(mshgrp, member);
+
+ return member;
+}
+
+/**
+ * @brief Update share group member's subscribed topic names.
+ *
+ * @param member The member to update.
+ * @param SubscribedTopicNames Array of topic names.
+ * @param SubscribedTopicNamesCnt Count of topic names:
+ * -1 = unchanged (no modification)
+ * 0 = clear all subscriptions
+ * >0 = set to provided topics
+ *
+ * @returns rd_true if subscriptions changed, rd_false otherwise.
+ */
+rd_bool_t rd_kafka_mock_sharegroup_member_subscribed_topic_names_set(
+ rd_kafka_mock_sharegroup_member_t *member,
+ const rd_kafkap_str_t *SubscribedTopicNames,
+ int32_t SubscribedTopicNamesCnt) {
+ int32_t i;
+
+ if (SubscribedTopicNamesCnt < 0) {
+ /* -1 means unchanged */
+ return rd_false;
+ }
+
+ if (SubscribedTopicNamesCnt == 0) {
+ /* 0 means clear all subscriptions */
+ if (!member->subscribed_topic_names ||
+ rd_list_cnt(member->subscribed_topic_names) == 0) {
+ /* Already empty, no change */
+ return rd_false;
+ }
+ rd_list_destroy(member->subscribed_topic_names);
+ member->subscribed_topic_names = NULL;
+ return rd_true;
+ }
+
+ /* SubscribedTopicNamesCnt > 0: Check if subscription changed */
+ if (member->subscribed_topic_names) {
+ if (rd_list_cnt(member->subscribed_topic_names) ==
+ SubscribedTopicNamesCnt) {
+ rd_bool_t same = rd_true;
+ char *topic;
+ int j;
+
+ RD_LIST_FOREACH(topic, member->subscribed_topic_names,
+ j) {
+ rd_bool_t found = rd_false;
+ for (i = 0; i < SubscribedTopicNamesCnt; i++) {
+ if (!rd_kafkap_str_cmp_str(
+ &SubscribedTopicNames[i],
+ topic)) {
+ found = rd_true;
+ break;
+ }
+ }
+ if (!found) {
+ same = rd_false;
+ break;
+ }
+ }
+ if (same)
+ return rd_false;
+ }
+ }
+
+ /* Subscription changed, update the list */
+ RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy);
+ member->subscribed_topic_names =
+ rd_list_new(SubscribedTopicNamesCnt, rd_free);
+
+ for (i = 0; i < SubscribedTopicNamesCnt; i++) {
+ rd_list_add(member->subscribed_topic_names,
+ RD_KAFKAP_STR_DUP(&SubscribedTopicNames[i]));
+ }
+
+ return rd_true;
+}
+
+/**
+ * @brief Collect all subscribed topic names from all members.
+ */
+static rd_list_t *rd_kafka_mock_sharegroup_collect_subscribed_topics(
+ rd_kafka_mock_sharegroup_t *mshgrp) {
+ rd_kafka_mock_sharegroup_member_t *member;
+ rd_list_t *all_topics;
+
+ all_topics = rd_list_new(32, rd_free);
+
+ TAILQ_FOREACH(member, &mshgrp->members, link) {
+ const char *topic;
+ int i;
+
+ if (!member->subscribed_topic_names)
+ continue;
+
+ RD_LIST_FOREACH(topic, member->subscribed_topic_names, i) {
+ const char *existing;
+ int j;
+ rd_bool_t found = rd_false;
+
+ /* Check if topic already in all_topics */
+ RD_LIST_FOREACH(existing, all_topics, j) {
+ if (!strcmp(topic, existing)) {
+ found = rd_true;
+ break;
+ }
+ }
+
+ /* Add if not found */
+ if (!found) {
+ rd_list_add(all_topics, rd_strdup(topic));
+ }
+ }
+ }
+
+ return all_topics;
+}
+
+/**
+ * @brief Get list of member ID's subscribed to a topic.
+ */
+rd_list_t *rd_kafka_mock_sharegroup_get_members_for_topic(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ char *topic_name) {
+ rd_kafka_mock_sharegroup_member_t *member;
+ rd_list_t *subscribed_members;
+ int member_idx = 0;
+
+ subscribed_members = rd_list_new(mshgrp->member_cnt, rd_free);
+
+ TAILQ_FOREACH(member, &mshgrp->members, link) {
+ char *topic;
+ int i;
+
+ if (member->subscribed_topic_names) {
+ RD_LIST_FOREACH(topic, member->subscribed_topic_names,
+ i) {
+ if (!strcmp(topic, topic_name)) {
+ int *idx = rd_malloc(sizeof(*idx));
+ *idx = member_idx;
+ rd_list_add(subscribed_members, idx);
+ break;
+ }
+ }
+ }
+ member_idx++;
+ }
+
+ return subscribed_members;
+}
+
+/**
+ * @brief Assign partitions of a single topic to subscribed members.
+ */
+void rd_kafka_mock_sharegroup_assign_topic_partitions(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_kafka_mock_topic_t *mtopic,
+ rd_list_t *subscribed_member_indices) {
+ int member_count;
+ int partition_cnt;
+ int partitions_per_member;
+ int extra_partitions;
+ int partition_idx;
+ int i;
+
+ member_count = rd_list_cnt(subscribed_member_indices);
+ partition_cnt = mtopic->partition_cnt;
+
+ if (member_count == 0 || partition_cnt == 0)
+ return;
+
+ partitions_per_member = partition_cnt / member_count;
+ extra_partitions = partition_cnt % member_count;
+ partition_idx = 0;
+
+ for (i = 0; i < member_count; i++) {
+ int *member_idx_ptr =
+ (int *)rd_list_elem(subscribed_member_indices, i);
+ rd_kafka_mock_sharegroup_member_t *member;
+ int j, cnt = 0;
+ int num_partitions;
+
+ TAILQ_FOREACH(member, &mshgrp->members, link) {
+ if (cnt == *member_idx_ptr)
+ break;
+ cnt++;
+ }
+
+ if (!member)
+ continue;
+
+ num_partitions =
+ partitions_per_member + (i < extra_partitions ? 1 : 0);
+
+ if (!member->assignment)
+ member->assignment =
+ rd_kafka_topic_partition_list_new(num_partitions);
+
+ for (j = 0; j < num_partitions && partition_idx < partition_cnt;
+ j++, partition_idx++) {
+ rd_kafka_topic_partition_t *rktpar;
+ rktpar = rd_kafka_topic_partition_list_add(
+ member->assignment, mtopic->name, partition_idx);
+ /* Set topic ID so the response can include it */
+ rd_kafka_topic_partition_set_topic_id(rktpar,
+ mtopic->id);
+ }
+ }
+}
+
+/**
+ * @brief Recalculate assignments for all members in the share group.
+ */
+void rd_kafka_mock_sharegroup_assignment_recalculate(
+ rd_kafka_mock_sharegroup_t *mshgrp) {
+ rd_kafka_mock_sharegroup_member_t *member;
+ rd_list_t *all_topics;
+ char *topic_name;
+ int i;
+
+ if (mshgrp->member_cnt == 0)
+ return;
+
+ /* Skip automatic assignment if manual mode is enabled */
+ if (mshgrp->manual_assignment)
+ return;
+
+ TAILQ_FOREACH(member, &mshgrp->members, link) {
+ if (member->assignment) {
+ rd_kafka_topic_partition_list_destroy(
+ member->assignment);
+ member->assignment = NULL;
+ }
+ }
+
+ all_topics = rd_kafka_mock_sharegroup_collect_subscribed_topics(mshgrp);
+
+ RD_LIST_FOREACH(topic_name, all_topics, i) {
+ rd_kafka_mock_topic_t *mtopic;
+ rd_list_t *subscribed_members;
+
+ mtopic = rd_kafka_mock_topic_find(mshgrp->cluster, topic_name);
+ if (!mtopic)
+ continue;
+
+ subscribed_members =
+ rd_kafka_mock_sharegroup_get_members_for_topic(mshgrp,
+ topic_name);
+
+ rd_kafka_mock_sharegroup_assign_topic_partitions(
+ mshgrp, mtopic, subscribed_members);
+
+ rd_list_destroy(subscribed_members);
+ }
+
+ mshgrp->group_epoch++;
+
+ TAILQ_FOREACH(member, &mshgrp->members, link) {
+ /* Save the current epoch as previous before bumping.
+ * This allows the client to catch up if the response
+ * with the new epoch was lost. */
+ member->previous_member_epoch = member->member_epoch;
+ member->member_epoch = mshgrp->group_epoch;
+ }
+
+ rd_list_destroy(all_topics);
+}
+
+/**
+ * @brief Create a new target assignment (manual)
+ */
+rd_kafka_mock_sharegroup_target_assignment_t *
+rd_kafka_mock_sharegroup_target_assignment_new(void) {
+ rd_kafka_mock_sharegroup_target_assignment_t *target_assignment;
+ target_assignment = rd_calloc(1, sizeof(*target_assignment));
+ rd_list_init(&target_assignment->member_ids, 0, rd_free);
+ rd_list_init(&target_assignment->assignment, 0,
+ (void *)rd_kafka_topic_partition_list_destroy);
+
+ return target_assignment;
+}
+
+/**
+ * @brief Destroy target assignment
+ */
+void rd_kafka_mock_sharegroup_target_assignment_destroy(
+ rd_kafka_mock_sharegroup_target_assignment_t *target_assignment) {
+ rd_list_destroy(&target_assignment->member_ids);
+ rd_list_destroy(&target_assignment->assignment);
+ rd_free(target_assignment);
+}
+
+/**
+ * @brief Set the target assignment for the sharegroup.
+ * This applies the manual assignment to the members.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_sharegroup_target_assignment_set(
+ rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_kafka_mock_sharegroup_target_assignment_t *target_assignment) {
+ rd_kafka_mock_sharegroup_member_t *member;
+ size_t i;
+
+ for (i = 0; i < rd_list_cnt(&target_assignment->member_ids); i++) {
+ const char *member_id =
+ rd_list_elem(&target_assignment->member_ids, i);
+ const rd_kafka_topic_partition_list_t *partitions =
+ rd_list_elem(&target_assignment->assignment, i);
+ rd_kafkap_str_t *member_id_str;
+
+ member_id_str = rd_kafkap_str_new(member_id, -1);
+ member =
+ rd_kafka_mock_sharegroup_member_find(mshgrp, member_id_str);
+ rd_kafkap_str_destroy(member_id_str);
+
+ if (!member) {
+ rd_kafka_dbg(mshgrp->cluster->rk, MOCK, "MOCK",
+ "Cannot set target assignment for "
+ "non-existing member %s in sharegroup %s",
+ member_id, mshgrp->id);
+ continue;
+ }
+
+ if (member->assignment) {
+ rd_kafka_topic_partition_list_destroy(
+ member->assignment);
+ }
+
+ member->assignment =
+ rd_kafka_topic_partition_list_copy(partitions);
+
+ /* Set topic IDs on each partition so the heartbeat response
+ * can include them (ShareGroupHeartbeat uses topic IDs) */
+ {
+ int j;
+ for (j = 0; j < member->assignment->cnt; j++) {
+ rd_kafka_topic_partition_t *rktpar =
+ &member->assignment->elems[j];
+ rd_kafkap_str_t topic_str = {
+ .str = rktpar->topic,
+ .len = strlen(rktpar->topic)};
+ rd_kafka_mock_topic_t *mtopic =
+ rd_kafka_mock_topic_find_by_kstr(
+ mshgrp->cluster, &topic_str);
+ if (mtopic) {
+ rd_kafka_topic_partition_set_topic_id(
+ rktpar, mtopic->id);
+ }
+ }
+ }
+
+ rd_kafka_dbg(
+ mshgrp->cluster->rk, MOCK, "MOCK",
+ "Target assignment set for member %s: %d partition(s)",
+ member_id, member->assignment->cnt);
+ }
+
+ /* Bump the epochs */
+ TAILQ_FOREACH(member, &mshgrp->members, link) {
+ member->previous_member_epoch = member->member_epoch;
+ member->member_epoch = ++mshgrp->group_epoch;
+ }
+}
+
+/**
+ * @brief Manual target assignment interface for sharegroups.
+ */
+void rd_kafka_mock_sharegroup_target_assignment(
+ rd_kafka_mock_cluster_t *mcluster,
+ const char *group_id,
+ const char **member_ids,
+ rd_kafka_topic_partition_list_t **assignment,
+ size_t member_cnt) {
+ rd_kafka_mock_sharegroup_t *mshgrp;
+ rd_kafka_mock_sharegroup_target_assignment_t *target_assignment;
+ size_t i;
+ rd_kafkap_str_t *group_id_str;
+
+ mtx_lock(&mcluster->lock);
+ group_id_str = rd_kafkap_str_new(group_id, -1);
+ mshgrp = rd_kafka_mock_sharegroup_find(mcluster, group_id_str);
+ rd_kafkap_str_destroy(group_id_str);
+
+ if (!mshgrp) {
+ rd_kafka_log(mcluster->rk, LOG_ERR, "MOCK",
+ "Sharegroup %s not found for target assignment",
+ group_id);
+ mtx_unlock(&mcluster->lock);
+ return;
+ }
+
+ mshgrp->manual_assignment = rd_true;
+ target_assignment = rd_kafka_mock_sharegroup_target_assignment_new();
+
+ for (i = 0; i < member_cnt; i++) {
+ rd_list_add(&target_assignment->member_ids,
+ rd_strdup(member_ids[i]));
+ rd_list_add(&target_assignment->assignment,
+ rd_kafka_topic_partition_list_copy(assignment[i]));
+ }
+ rd_kafka_mock_sharegroup_target_assignment_set(mshgrp,
+ target_assignment);
+ rd_kafka_mock_sharegroup_target_assignment_destroy(target_assignment);
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief Set the sharegroup session timeout for the sharegroup.
+ */
+void rd_kafka_mock_sharegroup_set_session_timeout(
+ rd_kafka_mock_cluster_t *mcluster,
+ int session_timeout_ms) {
+ mtx_lock(&mcluster->lock);
+ mcluster->defaults.sharegroup_session_timeout_ms = session_timeout_ms;
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief Set the sharegroup heartbeat interval for the sharegroup.
+ */
+void rd_kafka_mock_sharegroup_set_heartbeat_interval(
+ rd_kafka_mock_cluster_t *mcluster,
+ int heartbeat_interval_ms) {
+ mtx_lock(&mcluster->lock);
+ mcluster->defaults.sharegroup_heartbeat_interval_ms =
+ heartbeat_interval_ms;
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief Set the maximum delivery attempts per record for the sharegroup.
+ */
+void rd_kafka_mock_sharegroup_set_max_delivery_attempts(
+ rd_kafka_mock_cluster_t *mcluster,
+ int max_attempts) {
+ rd_kafka_mock_sharegroup_t *mshgrp;
+ mtx_lock(&mcluster->lock);
+ TAILQ_FOREACH(mshgrp, &mcluster->sharegrps, link)
+ mshgrp->max_delivery_attempts = max_attempts;
+ mcluster->defaults.sharegroup_max_delivery_attempts = max_attempts;
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief Set the per-record lock duration in milliseconds for the sharegroup.
+ */
+void rd_kafka_mock_sharegroup_set_record_lock_duration(
+ rd_kafka_mock_cluster_t *mcluster,
+ int lock_duration_ms) {
+ rd_kafka_mock_sharegroup_t *mshgrp;
+ mtx_lock(&mcluster->lock);
+ TAILQ_FOREACH(mshgrp, &mcluster->sharegrps, link)
+ mshgrp->record_lock_duration_ms = lock_duration_ms;
+ mcluster->defaults.sharegroup_record_lock_duration_ms =
+ lock_duration_ms;
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief Destroy share fetch session.
+ */
+void rd_kafka_mock_sgrp_fetch_session_destroy(
+ rd_kafka_mock_sgrp_fetch_session_t *session) {
+ rd_free(session->member_id);
+ RD_IF_FREE(session->partitions, rd_kafka_topic_partition_list_destroy);
+ rd_free(session);
+}
+
+/**
+ * @brief Release all ACQUIRED records owned by \p member_id across all
+ * share-partition metadata in the share group.
+ *
+ * This is called when a session is closed (epoch=-1), when a session
+ * times out, or as part of periodic lock-expiry reclamation.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+void rd_kafka_mock_sgrp_release_member_locks(rd_kafka_mock_sharegroup_t *mshgrp,
+ const char *member_id) {
+ rd_kafka_mock_sgrp_partmeta_t *pmeta;
+
+ TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) {
+ rd_kafka_mock_sgrp_record_state_t *state, *tmp;
+ TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) {
+ if (state->state != RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED)
+ continue;
+ if (!state->owner_member_id)
+ continue;
+ if (strcmp(state->owner_member_id, member_id) != 0)
+ continue;
+
+ state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE;
+ rd_free(state->owner_member_id);
+ state->owner_member_id = NULL;
+ state->lock_expiry_ts = 0;
+ }
+ }
+}
+
+/**
+ * @brief Proactively release any expired acquisition locks.
+ *
+ * Iterates all partition metadata in the share group and flips
+ * ACQUIRED records whose lock_expiry_ts has passed back to AVAILABLE.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+static void rd_kafka_mock_sgrp_expire_locks(rd_kafka_mock_sharegroup_t *mshgrp,
+ rd_ts_t now) {
+ rd_kafka_mock_sgrp_partmeta_t *pmeta;
+
+ TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) {
+ rd_kafka_mock_sgrp_record_state_t *state, *tmp;
+ TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) {
+ if (state->state != RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED)
+ continue;
+ if (!state->lock_expiry_ts ||
+ state->lock_expiry_ts > now)
+ continue;
+
+ /* Lock has expired — release back to AVAILABLE */
+ state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE;
+ RD_IF_FREE(state->owner_member_id, rd_free);
+ state->owner_member_id = NULL;
+ state->lock_expiry_ts = 0;
+ }
+ }
+}
+
+/**
+ * @brief Periodic timer: expire stale share-fetch sessions and
+ * proactively reclaim expired acquisition locks.
+ *
+ * @locks mcluster->lock is acquired and released.
+ */
+void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts,
+ void *arg) {
+ rd_kafka_mock_sharegroup_t *mshgrp = arg;
+ rd_kafka_mock_sgrp_fetch_session_t *session, *tmp;
+ rd_ts_t now = rd_clock();
+ rd_kafka_mock_cluster_t *mcluster = mshgrp->cluster;
+
+ (void)rkts;
+
+ mtx_lock(&mcluster->lock);
+
+ /* 1. Expire stale sessions and release their member locks. */
+ TAILQ_FOREACH_SAFE(session, &mshgrp->fetch_sessions, link, tmp) {
+ if (session->ts_last_activity +
+ (mshgrp->session_timeout_ms * 1000) >
+ now)
+ continue;
+
+ /* Release all locks held by this member before
+ * destroying the session. */
+ rd_kafka_mock_sgrp_release_member_locks(mshgrp,
+ session->member_id);
+
+ TAILQ_REMOVE(&mshgrp->fetch_sessions, session, link);
+ mshgrp->fetch_session_cnt--;
+ rd_kafka_mock_sgrp_fetch_session_destroy(session);
+ }
+
+ /* 2. Proactively reclaim any expired acquisition locks.
+ * This catches records whose owning consumer crashed
+ * without closing its session cleanly. */
+ rd_kafka_mock_sgrp_expire_locks(mshgrp, now);
+
+ mtx_unlock(&mcluster->lock);
+}
+
+/**
+ * @brief A client connection closed, check if any sharegroup has any
+ * state for this connection that needs to be cleared.
+ *
+ * @param mcluster Cluster to search in.
+ * @param mconn Connection that was closed.
+ *
+ * @locks mcluster->lock MUST be held.
+ */
+void rd_kafka_mock_sharegrps_connection_closed(
+ rd_kafka_mock_cluster_t *mcluster,
+ rd_kafka_mock_connection_t *mconn) {
+ rd_kafka_mock_sharegroup_t *mshgrp;
+
+ TAILQ_FOREACH(mshgrp, &mcluster->sharegrps, link) {
+ rd_kafka_mock_sharegroup_member_t *member, *tmp;
+ TAILQ_FOREACH_SAFE(member, &mshgrp->members, link, tmp) {
+ if (member->conn == mconn) {
+ rd_kafka_mock_sharegroup_member_fenced(mshgrp,
+ member);
+ }
+ }
+ }
+}
+
+/**
+ * @brief Retrieve the member IDs from a sharegroup.
+ *
+ * @param mcluster Mock cluster instance.
+ * @param group_id The sharegroup ID.
+ * @param member_ids_out Output array of member IDs (caller must free each
+ * string and the array itself).
+ * @param member_cnt_out Output count of members.
+ *
+ * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success,
+ * RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND if sharegroup not found.
+ */
+rd_kafka_resp_err_t
+rd_kafka_mock_sharegroup_get_member_ids(rd_kafka_mock_cluster_t *mcluster,
+ const char *group_id,
+ char ***member_ids_out,
+ size_t *member_cnt_out) {
+ rd_kafka_mock_sharegroup_t *mshgrp;
+ rd_kafka_mock_sharegroup_member_t *member;
+ rd_kafkap_str_t *group_id_str;
+ char **member_ids;
+ size_t i;
+
+ mtx_lock(&mcluster->lock);
+ group_id_str = rd_kafkap_str_new(group_id, -1);
+ mshgrp = rd_kafka_mock_sharegroup_find(mcluster, group_id_str);
+ rd_kafkap_str_destroy(group_id_str);
+
+ if (!mshgrp) {
+ mtx_unlock(&mcluster->lock);
+ *member_ids_out = NULL;
+ *member_cnt_out = 0;
+ return RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND;
+ }
+
+ *member_cnt_out = mshgrp->member_cnt;
+ if (mshgrp->member_cnt == 0) {
+ mtx_unlock(&mcluster->lock);
+ *member_ids_out = NULL;
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+ }
+
+ member_ids = rd_malloc(sizeof(*member_ids) * mshgrp->member_cnt);
+ i = 0;
+ TAILQ_FOREACH(member, &mshgrp->members, link) {
+ member_ids[i++] = rd_strdup(member->id);
+ }
+
+ mtx_unlock(&mcluster->lock);
+ *member_ids_out = member_ids;
+ return RD_KAFKA_RESP_ERR_NO_ERROR;
+}
\ No newline at end of file
diff --git a/src/rdkafka_msg.h b/src/rdkafka_msg.h
index 063fe96b95..8775f2be19 100644
--- a/src/rdkafka_msg.h
+++ b/src/rdkafka_msg.h
@@ -154,6 +154,11 @@ typedef struct rd_kafka_msg_s {
* protocol msg */
int32_t leader_epoch; /**< Leader epoch at the time
* the message was fetched. */
+ int8_t
+ ack_type; /**< Share consumer: acknowledgement
+ * type
+ * (rd_kafka_share_internal_acknowledgement_type).
+ * Set during response processing. */
} consumer;
} rkm_u;
} rd_kafka_msg_t;
@@ -230,7 +235,7 @@ typedef struct rd_kafka_msgq_s {
} rd_kafka_msgq_t;
#define RD_KAFKA_MSGQ_INITIALIZER(rkmq) \
- { .rkmq_msgs = TAILQ_HEAD_INITIALIZER((rkmq).rkmq_msgs) }
+ {.rkmq_msgs = TAILQ_HEAD_INITIALIZER((rkmq).rkmq_msgs)}
#define RD_KAFKA_MSGQ_FOREACH(elm, head) \
TAILQ_FOREACH(elm, &(head)->rkmq_msgs, rkm_link)
diff --git a/src/rdkafka_msgset.h b/src/rdkafka_msgset.h
index ee897b35bd..14393bff71 100644
--- a/src/rdkafka_msgset.h
+++ b/src/rdkafka_msgset.h
@@ -78,6 +78,13 @@ rd_kafka_msgset_parse(rd_kafka_buf_t *rkbuf,
rd_kafka_aborted_txns_t *aborted_txns,
const struct rd_kafka_toppar_ver *tver);
+rd_kafka_resp_err_t
+rd_kafka_share_msgset_parse(rd_kafka_buf_t *rkbuf,
+ rd_kafka_toppar_t *rktp,
+ rd_kafka_aborted_txns_t *aborted_txns,
+ const struct rd_kafka_toppar_ver *tver,
+ rd_kafka_q_t *par_rkq);
+
#if WITH_ZLIB
rd_kafka_resp_err_t rd_kafka_gzip_compress(rd_kafka_broker_t *rkb,
int comp_level,
diff --git a/src/rdkafka_msgset_reader.c b/src/rdkafka_msgset_reader.c
index 451dd35442..848a4cfe45 100644
--- a/src/rdkafka_msgset_reader.c
+++ b/src/rdkafka_msgset_reader.c
@@ -1422,6 +1422,9 @@ 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
@@ -1498,6 +1501,40 @@ rd_kafka_msgset_parse(rd_kafka_buf_t *rkbuf,
}
+/**
+ * @brief Parse one MessageSet at the current buffer read position,
+ * enqueueing messages, propagating errors, etc.
+ * @remark The current rkbuf_reader slice must be limited to the MessageSet size
+ *
+ * @returns see rd_kafka_msgset_reader_run()
+ */
+rd_kafka_resp_err_t
+rd_kafka_share_msgset_parse(rd_kafka_buf_t *rkbuf,
+ rd_kafka_toppar_t *rktp,
+ rd_kafka_aborted_txns_t *aborted_txns,
+ const struct rd_kafka_toppar_ver *tver,
+ rd_kafka_q_t *par_rkq) {
+ rd_kafka_msgset_reader_t msetr;
+ rd_kafka_resp_err_t err;
+
+ rd_kafka_msgset_reader_init(&msetr, rkbuf, rktp, tver, aborted_txns,
+ par_rkq);
+
+ /* Parse and handle the message set */
+ err = rd_kafka_msgset_reader_run(&msetr);
+
+ rd_atomic64_add(&rktp->rktp_c.rx_msgs, msetr.msetr_msgcnt);
+ rd_atomic64_add(&rktp->rktp_c.rx_msg_bytes, msetr.msetr_msg_bytes);
+
+ rd_avg_add(&rktp->rktp_rkt->rkt_avg_batchcnt,
+ (int64_t)msetr.msetr_msgcnt);
+ rd_avg_add(&rktp->rktp_rkt->rkt_avg_batchsize,
+ (int64_t)msetr.msetr_msg_bytes);
+
+ return err;
+}
+
+
/**
* @brief Offset comparator
*/
diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c
index 5dbbf9c9d4..5a920cd4eb 100644
--- a/src/rdkafka_op.c
+++ b/src/rdkafka_op.c
@@ -122,7 +122,14 @@ 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",
+ [RD_KAFKA_OP_SHARE_FETCH_RESPONSE] = "REPLY:SHARE_FETCH_RESPONSE",
};
if (type & RD_KAFKA_OP_REPLY)
@@ -287,6 +294,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,
+ [RD_KAFKA_OP_SHARE_FETCH_RESPONSE] =
+ sizeof(rko->rko_u.share_fetch_response),
};
size_t tsize = op2size[type & ~RD_KAFKA_OP_FLAGMASK];
@@ -507,6 +521,53 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) {
rd_kafka_broker_destroy);
break;
+ case RD_KAFKA_OP_SHARE_FETCH: {
+ rd_kafka_share_ack_batches_t *batch;
+ int i;
+ RD_IF_FREE(rko->rko_u.share_fetch.target_broker,
+ rd_kafka_broker_destroy);
+ if (rko->rko_u.share_fetch.ack_details) {
+ RD_LIST_FOREACH(batch,
+ rko->rko_u.share_fetch.ack_details, i) {
+ rd_kafka_share_ack_batches_destroy(batch,
+ rd_true);
+ }
+ rd_list_destroy(rko->rko_u.share_fetch.ack_details);
+ }
+ break;
+ }
+
+ case RD_KAFKA_OP_SHARE_FETCH_FANOUT:
+ /* Destroy ack_batches list if not already transferred.
+ * rd_list_destroy() frees the struct (RD_LIST_F_ALLOCATED). */
+ if (rko->rko_u.share_fetch_fanout.ack_batches)
+ rd_list_destroy(
+ rko->rko_u.share_fetch_fanout.ack_batches);
+ break;
+
+ case RD_KAFKA_OP_SHARE_FETCH_RESPONSE: {
+ rd_kafka_op_t *msg_rko;
+ int i;
+
+ /* Destroy all message ops in the list */
+ if (rko->rko_u.share_fetch_response.message_rkos) {
+ RD_LIST_FOREACH(
+ msg_rko,
+ rko->rko_u.share_fetch_response.message_rkos, i) {
+ rd_kafka_op_destroy(msg_rko);
+ }
+ rd_list_destroy_free(
+ rko->rko_u.share_fetch_response.message_rkos);
+ }
+
+ /* inflight_acks batches were transferred to rkshare in
+ * rd_kafka_share_build_ack_mapping(); only destroy the list. */
+ if (rko->rko_u.share_fetch_response.inflight_acks)
+ rd_list_destroy_free(
+ rko->rko_u.share_fetch_response.inflight_acks);
+ break;
+ }
+
default:
break;
}
diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h
index e79309aa02..b7169ad615 100644
--- a/src/rdkafka_op.h
+++ b/src/rdkafka_op.h
@@ -189,6 +189,17 @@ 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_SHARE_FETCH_RESPONSE, /**< Share fetch response containing
+ * all messages and partition acks
+ * from a single broker response. */
+
RD_KAFKA_OP__END
} rd_kafka_op_type_t;
@@ -607,7 +618,8 @@ struct rd_kafka_op_s {
* PART_PUSH_LEADER_RESPONSE
*/
char *str; /**< For:
- * COORD_SET (key) */
+ * COORD_SET (key)
+ */
int32_t partition; /**< For:
* PART_SET_FOLLOWER
* PART_SET_FOLLOWER_WMARKS
@@ -724,6 +736,82 @@ 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;
+
+ /** Whether records were fetched in this
+ * share-fetch response. Set by broker
+ * thread, read by main thread in reply
+ * handler. */
+ rd_bool_t records_fetched;
+
+ /** Ack batches to send with this request.
+ * Type: rd_kafka_share_ack_batches_t*.
+ * Moved from rkb_share_async_ack_details
+ * when creating the op. Freed by broker
+ * thread after use. */
+ rd_list_t *ack_details;
+ } share_fetch;
+
+ struct {
+ /** Whether this FANOUT should fetch more records.
+ * When rd_true, the selected broker's SHARE_FETCH
+ * op will have should_fetch=true.
+ * When rd_false, this is an ack-only FANOUT and
+ * no broker will fetch new records. */
+ rd_bool_t fetch_more_records;
+
+ /** List of all acknowledgement batches to send.
+ * Type: rd_kafka_share_ack_batches_t*
+ * Built from inflight ack map, will be filtered
+ * by leader when creating SHARE_FETCH ops.
+ * Each entry uses size=1 with types[0] holding the
+ * single ack type for the collated range.
+ * Set to NULL after ownership is transferred
+ * to per-broker ack_details.
+ */
+ rd_list_t *ack_batches;
+ } share_fetch_fanout;
+
+ /**
+ * Share fetch response - single rko containing all messages
+ * and partition ack info from one broker response.
+ */
+ struct {
+ /** List of message ops (rd_kafka_op_t*).
+ * Contains only actual messages (ACQUIRED/REJECT),
+ * no GAP placeholder ops.
+ */
+ /*
+ * TODO KIP-932: Check if we can send the messages only
+ * instead of the message rkos.
+ */
+ rd_list_t *message_rkos;
+
+ /** List of per-partition inflight ack mappings.
+ * Type: rd_kafka_share_ack_batches_t*
+ * Contains per-offset ack types (ACQUIRED/GAP/REJECT).
+ * Built in broker thread, merged to rkshare in app
+ * thread.
+ */
+ rd_list_t *inflight_acks;
+ } share_fetch_response;
+
} rko_u;
};
diff --git a/src/rdkafka_partition.c b/src/rdkafka_partition.c
index ce4f01b467..606aebaf9e 100644
--- a/src/rdkafka_partition.c
+++ b/src/rdkafka_partition.c
@@ -2594,6 +2594,32 @@ 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;
+}
+
+/**
+ * @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..f3609aa170 100644
--- a/src/rdkafka_partition.h
+++ b/src/rdkafka_partition.h
@@ -32,6 +32,8 @@
#include "rdkafka_topic.h"
#include "rdkafka_cgrp.h"
#include "rdkafka_broker.h"
+#include "rdkafka_share_acknowledgement.h"
+#include
extern const char *rd_kafka_fetch_states[];
@@ -131,6 +133,7 @@ rd_kafka_fetch_pos_make(int64_t offset,
typedef TAILQ_HEAD(rd_kafka_toppar_tqhead_s,
rd_kafka_toppar_s) rd_kafka_toppar_tqhead_t;
+
/**
* Topic + Partition combination
*/
@@ -142,9 +145,12 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */
TAILQ_ENTRY(rd_kafka_toppar_s) rktp_rktlink; /* rd_kafka_topic_t link*/
TAILQ_ENTRY(rd_kafka_toppar_s) rktp_cgrplink; /* rd_kafka_cgrp_t link */
TAILQ_ENTRY(rd_kafka_toppar_s)
- rktp_txnlink; /**< rd_kafka_t.rk_eos.
- * txn_pend_rktps
- * or txn_rktps */
+ 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()
@@ -179,10 +185,12 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */
int rktp_fetch; /* On rkb_active_toppars list */
/* Consumer */
- rd_kafka_q_t *rktp_fetchq; /* Queue of fetched messages
- * from broker.
- * Broker thread -> App */
- rd_kafka_q_t *rktp_ops; /* * -> Main thread */
+ rd_kafka_q_t *rktp_fetchq; /* Queue of fetched messages
+ * from broker.
+ * Broker thread -> App */
+ rd_kafka_q_t *rktp_temp_fetchq; /* Temporary fetch queue
+ * used to filter acquired records */
+ rd_kafka_q_t *rktp_ops; /* * -> Main thread */
rd_atomic32_t rktp_msgs_inflight; /**< Current number of
* messages in-flight to/from
@@ -478,6 +486,7 @@ struct rd_kafka_toppar_s { /* rd_kafka_toppar_t */
rd_atomic64_t rx_ver_drops; /**< Consumer: outdated message
* drops. */
} rktp_c;
+
};
/**
@@ -533,10 +542,22 @@ 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 +698,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);
+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);
@@ -767,6 +792,7 @@ int rd_kafka_topic_partition_match(rd_kafka_t *rk,
int rd_kafka_topic_partition_cmp(const void *_a, const void *_b);
int rd_kafka_topic_partition_by_id_cmp(const void *_a, const void *_b);
unsigned int rd_kafka_topic_partition_hash(const void *a);
+unsigned int rd_kafka_topic_partition_hash_by_id(const void *a);
int rd_kafka_topic_partition_list_find_idx(
const rd_kafka_topic_partition_list_t *rktparlist,
diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h
index 02565ecb3b..7fa0c21bc8 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];
@@ -281,11 +283,9 @@ typedef struct rd_kafkap_str_s {
/* strndup() a Kafka string */
#define RD_KAFKAP_STR_DUP(kstr) rd_strndup((kstr)->str, RD_KAFKAP_STR_LEN(kstr))
-#define RD_KAFKAP_STR_INITIALIZER \
- { .len = RD_KAFKAP_STR_LEN_NULL, .str = NULL }
+#define RD_KAFKAP_STR_INITIALIZER {.len = RD_KAFKAP_STR_LEN_NULL, .str = NULL}
-#define RD_KAFKAP_STR_INITIALIZER_EMPTY \
- { .len = 0, .str = "" }
+#define RD_KAFKAP_STR_INITIALIZER_EMPTY {.len = 0, .str = ""}
/**
* Frees a Kafka string previously allocated with `rd_kafkap_str_new()`
*/
@@ -661,8 +661,7 @@ typedef struct rd_kafka_pid_s {
int16_t epoch; /**< Producer Epoch */
} rd_kafka_pid_t;
-#define RD_KAFKA_PID_INITIALIZER \
- { -1, -1 }
+#define RD_KAFKA_PID_INITIALIZER {-1, -1}
/**
* @returns true if \p PID is valid
diff --git a/src/rdkafka_protocol.h b/src/rdkafka_protocol.h
index 19190e1447..ddbe04a9b7 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..f0cae3aa51 100644
--- a/src/rdkafka_queue.c
+++ b/src/rdkafka_queue.c
@@ -870,6 +870,182 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq,
}
+/**
+ * @brief Process a share fetch response op and deliver messages.
+ *
+ * @param rko The share fetch response op
+ * @param rkshare The share consumer handle
+ * @param rkmessages Output array for messages
+ * @param rkmessages_size Size of output array
+ * @param cnt Current count of messages in array
+ *
+ * @returns Updated count of messages in array
+ */
+static unsigned int
+rd_kafka_share_process_fetch_response(rd_kafka_op_t *rko,
+ rd_kafka_share_t *rkshare,
+ rd_kafka_message_t **rkmessages,
+ unsigned int cnt) {
+ rd_kafka_op_t *msg_rko;
+ int i;
+ int total_msgs =
+ rd_list_cnt(rko->rko_u.share_fetch_response.message_rkos);
+
+ /* Build acknowledgement mapping from inflight_acks */
+ rd_kafka_share_build_ack_mapping(rkshare, rko);
+
+ /* Process all messages from the list. */
+ for (i = 0; i < total_msgs; i++) {
+ msg_rko = rd_list_elem(
+ rko->rko_u.share_fetch_response.message_rkos, i);
+
+ /**
+ * TODO KIP-932: Check and update the handling of control
+ * messages
+ */
+ if (unlikely(rd_kafka_op_is_ctrl_msg(msg_rko)))
+ continue;
+
+ /* Return message to application */
+ rkmessages[cnt++] = rd_kafka_message_get(msg_rko);
+ }
+
+ return cnt;
+}
+
+/**
+ * TODO KIP-932: Update the handling of callbacks and other op types. Currently
+ * we are only enqueing RD_KAFKA_OP_SHARE_FETCH_RESPONSE and
+ * RD_KAFKA_OP_CONSUMER_ERR opps to the application thread.
+ */
+// static unsigned int rd_kafka_share_handle_other_op(rd_kafka_t *rk,
+// rd_kafka_q_t *rkq,
+// rd_kafka_op_t *rko,
+// rd_kafka_message_t
+// **rkmessages, unsigned int
+// cnt, rd_bool_t
+// *should_break) {
+// rd_kafka_op_res_t res;
+
+// *should_break = rd_false;
+
+// /* Handle outdated ops */
+// if (rd_kafka_op_version_outdated(rko, 0)) {
+// rd_kafka_op_destroy(rko);
+// return cnt;
+// }
+
+// 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)
+// return cnt;
+
+// if (unlikely(res == RD_KAFKA_OP_RES_YIELD || rd_kafka_yield_thread))
+// {
+// *should_break = rd_true;
+// return cnt;
+// }
+
+
+
+// return cnt;
+// }
+
+/**
+ * Serve all ops from the share consumer queue. Only CONSUMER_ERR and
+ * SHARE_FETCH_RESPONSE are enqueued. Processes all available ops:
+ * - On CONSUMER_ERR: return that error immediately
+ * - On SHARE_FETCH_RESPONSE: accumulate messages from all responses
+ */
+rd_kafka_error_t *
+rd_kafka_q_serve_share_rkmessages(rd_kafka_q_t *rkq,
+ int timeout_ms,
+ rd_kafka_message_t **rkmessages,
+ size_t rkmessages_size,
+ size_t *rkmessages_size_out) {
+ rd_kafka_op_t *rko;
+ rd_kafka_t *rk = rkq->rkq_rk;
+ rd_kafka_q_t *fwdq;
+ rd_ts_t abs_timeout;
+ rd_kafka_error_t *error = NULL;
+ unsigned int cnt = 0;
+
+ *rkmessages_size_out = 0;
+
+ mtx_lock(&rkq->rkq_lock);
+ if ((fwdq = rd_kafka_q_fwd_get(rkq, 0))) {
+ mtx_unlock(&rkq->rkq_lock);
+ error = rd_kafka_q_serve_share_rkmessages(
+ fwdq, timeout_ms, rkmessages, rkmessages_size,
+ rkmessages_size_out);
+ rd_kafka_q_destroy(fwdq);
+ return error;
+ }
+ 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;
+
+ /* Wait for at least one op to arrive */
+ mtx_lock(&rkq->rkq_lock);
+ while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) &&
+ !rd_kafka_q_check_yield(rkq) &&
+ 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);
+ rd_kafka_app_polled(rk, rkq);
+ return NULL;
+ }
+
+ /* Process all available ops in the queue */
+ while ((rko = TAILQ_FIRST(&rkq->rkq_q))) {
+ rd_kafka_q_deq0(rkq, rko);
+ mtx_unlock(&rkq->rkq_lock);
+
+ if (rko->rko_type == RD_KAFKA_OP_CONSUMER_ERR) {
+ /* Return error immediately */
+ if (rko->rko_error) {
+ error = rko->rko_error;
+ rko->rko_error = NULL;
+ } else {
+ error = rd_kafka_error_new(
+ rko->rko_err, "%s",
+ rko->rko_u.err.errstr
+ ? rko->rko_u.err.errstr
+ : "");
+ }
+ rd_kafka_op_destroy(rko);
+ rd_kafka_app_polled(rk, rkq);
+ *rkmessages_size_out = cnt;
+ return error;
+ }
+
+ if (rko->rko_type == RD_KAFKA_OP_SHARE_FETCH_RESPONSE) {
+ /* Accumulate messages from this response */
+ cnt = rd_kafka_share_process_fetch_response(
+ rko, rkq->rkq_rk->rk_rkshare, rkmessages, cnt);
+ rkq->rkq_rk->rk_rkshare
+ ->rkshare_fetch_more_records_requested = rd_false;
+ }
+
+ /* Destroy other op types */
+ if (rko->rko_type != RD_KAFKA_OP_SHARE_FETCH_RESPONSE)
+ rd_kafka_op_destroy(rko);
+
+ mtx_lock(&rkq->rkq_lock);
+ }
+ mtx_unlock(&rkq->rkq_lock);
+
+ rd_kafka_app_polled(rk, rkq);
+ *rkmessages_size_out = cnt;
+ return NULL;
+}
+
+
void rd_kafka_queue_destroy(rd_kafka_queue_t *rkqu) {
if (rkqu->rkqu_is_owner)
diff --git a/src/rdkafka_queue.h b/src/rdkafka_queue.h
index ff1a465770..b67ceabd39 100644
--- a/src/rdkafka_queue.h
+++ b/src/rdkafka_queue.h
@@ -878,6 +878,13 @@ int rd_kafka_q_serve_rkmessages(rd_kafka_q_t *rkq,
int timeout_ms,
rd_kafka_message_t **rkmessages,
size_t rkmessages_size);
+/** Returns error (caller must destroy) or NULL. */
+rd_kafka_error_t *
+rd_kafka_q_serve_share_rkmessages(rd_kafka_q_t *rkq,
+ int timeout_ms,
+ rd_kafka_message_t **rkmessages,
+ size_t rkmessages_size,
+ size_t *rkmessages_size_out);
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 d38101629d..5901e28b88 100644
--- a/src/rdkafka_request.c
+++ b/src/rdkafka_request.c
@@ -562,6 +562,9 @@ 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 +2445,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/src/rdkafka_share_acknowledgement.c b/src/rdkafka_share_acknowledgement.c
new file mode 100644
index 0000000000..5f39e30465
--- /dev/null
+++ b/src/rdkafka_share_acknowledgement.c
@@ -0,0 +1,480 @@
+/*
+ * librdkafka - The Apache Kafka C/C++ library
+ *
+ * Copyright (c) 2015-2022, Magnus Edenhill,
+ * 2023, 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 "rdkafka_int.h"
+#include "rdkafka_share_acknowledgement.h"
+
+rd_kafka_share_ack_batch_entry_t *
+rd_kafka_share_ack_batch_entry_new(int64_t start_offset,
+ int64_t end_offset,
+ int32_t types_cnt) {
+ rd_kafka_share_ack_batch_entry_t *entry;
+
+ entry = rd_calloc(1, sizeof(*entry));
+ entry->start_offset = start_offset;
+ entry->end_offset = end_offset;
+ entry->size = end_offset - start_offset + 1;
+ entry->types_cnt = types_cnt;
+ entry->delivery_count = 0;
+ entry->types = rd_calloc((size_t)types_cnt, sizeof(*entry->types));
+ return entry;
+}
+
+void rd_kafka_share_ack_batch_entry_destroy(
+ rd_kafka_share_ack_batch_entry_t *entry) {
+ if (!entry)
+ return;
+ rd_free(entry->types);
+ rd_free(entry);
+}
+
+void rd_kafka_share_ack_batches_destroy_free(void *ptr) {
+ rd_kafka_share_ack_batches_destroy((rd_kafka_share_ack_batches_t *)ptr,
+ rd_true);
+}
+
+rd_kafka_share_ack_batches_t *rd_kafka_share_ack_batches_new(void) {
+ rd_kafka_share_ack_batches_t *batches;
+
+ batches = rd_calloc(1, sizeof(*batches));
+ rd_list_init(&batches->entries, 0, NULL);
+ batches->rktpar = NULL;
+ batches->response_leader_id = 0;
+ batches->response_leader_epoch = 0;
+ batches->response_msgs_count = 0;
+ return batches;
+}
+
+void rd_kafka_share_ack_batches_destroy(rd_kafka_share_ack_batches_t *batches,
+ rd_bool_t free_rktpar) {
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int i;
+
+ if (!batches)
+ return;
+ RD_LIST_FOREACH(entry, &batches->entries, i)
+ rd_kafka_share_ack_batch_entry_destroy(entry);
+ rd_list_destroy(&batches->entries);
+ if (free_rktpar && batches->rktpar)
+ rd_kafka_topic_partition_destroy(batches->rktpar);
+ rd_free(batches);
+}
+
+/**
+ * @brief Transfer inflight acks from response RKO into rkshare's inflight map.
+ *
+ * Takes each batch from the response's inflight_acks list and stores it in
+ * the map (key = topic-partition). Ownership is transferred; the response
+ * RKO must not free these when destroyed.
+ * In the future, the map will be cleared per partition after acks are sent.
+ *
+ * @param rkshare Share consumer handle
+ * @param response_rko The share fetch response RKO containing inflight_acks
+ */
+void rd_kafka_share_build_ack_mapping(rd_kafka_share_t *rkshare,
+ rd_kafka_op_t *response_rko) {
+ rd_list_t *list =
+ response_rko->rko_u.share_fetch_response.inflight_acks;
+
+ while (rd_list_cnt(list) > 0) {
+ rd_kafka_share_ack_batches_t *batches = rd_list_pop(list);
+ rd_kafka_topic_partition_t *key;
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int i, k;
+
+ rd_dassert(batches->rktpar != NULL);
+
+ key = rd_kafka_topic_partition_new_with_topic_id(
+ rd_kafka_topic_partition_get_topic_id(batches->rktpar),
+ batches->rktpar->partition);
+
+ /* Each topic-partition is always a new entry (no overwrites).
+ */
+ RD_MAP_SET(&rkshare->rkshare_inflight_acks, key, batches);
+
+ /* Count ACQUIRED types for unacked tracking */
+ RD_LIST_FOREACH(entry, &batches->entries, i) {
+ for (k = 0; k < entry->types_cnt; k++) {
+ if (entry->types[k] ==
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED)
+ rkshare->rkshare_unacked_cnt++;
+ }
+ }
+ }
+}
+
+/**
+ * @brief Free a collated share ack batch and its entries.
+ *
+ * Note: Does NOT destroy rktpar as it's shared with the source inflight map.
+ */
+static void rd_kafka_share_collated_batch_destroy(void *ptr) {
+ rd_kafka_share_ack_batches_t *batch = ptr;
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int i;
+
+ if (!batch)
+ return;
+ RD_LIST_FOREACH(entry, &batch->entries, i)
+ rd_kafka_share_ack_batch_entry_destroy(entry);
+ rd_list_destroy(&batch->entries);
+ rd_free(batch);
+}
+
+/**
+ * @brief Convert ACQUIRED to ACCEPT for sending to broker.
+ *
+ * Broker expects ACCEPT (AVAILABLE) for offsets we're acknowledging;
+ * internally we use ACQUIRED until we send.
+ */
+static rd_kafka_share_internal_acknowledgement_type
+rd_kafka_share_ack_convert_acquired_to_accept(
+ rd_kafka_share_internal_acknowledgement_type type) {
+ if (type == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED)
+ return RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT;
+ return type;
+}
+
+/**
+ * @brief Create a new collated batch entry with a single acknowledgement type.
+ *
+ * Creates an entry where size=1 and types[0] holds the single ack type
+ * for the entire offset range [start_offset, end_offset].
+ *
+ * @param start_offset First offset in the range
+ * @param end_offset Last offset in the range (inclusive)
+ * @param type The acknowledgement type for the entire range
+ *
+ * @returns Newly allocated collated entry (caller must free)
+ */
+static rd_kafka_share_ack_batch_entry_t *
+rd_kafka_share_ack_batch_entry_collated_new(
+ int64_t start_offset,
+ int64_t end_offset,
+ rd_kafka_share_internal_acknowledgement_type type) {
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_kafka_share_ack_batch_entry_new(start_offset, end_offset, 1);
+ entry->types[0] = type;
+ return entry;
+}
+
+/**
+ * @brief Collate a batch with per-offset types into single-type entries.
+ *
+ * Takes an input batch where each entry has a types[] array with per-offset
+ * acknowledgement types, and produces an output batch where consecutive
+ * offsets with the same type are merged into single entries (types_cnt=1,
+ * types[0]=type, size=offset count).
+ *
+ * ACQUIRED type is converted to ACCEPT for sending to broker.
+ *
+ * @param src Source batch with per-offset types array
+ * @param dst Output batch to populate with collated entries
+ * (must be pre-initialized with rd_list_init on entries)
+ *
+ * Example: Input entry with types [ACQ, ACQ, GAP, REJ, REJ] for offsets 1-5
+ * produces output entries:
+ * - {start:1, end:2, size:1, types[0]:ACCEPT}
+ * - {start:3, end:3, size:1, types[0]:GAP}
+ * - {start:4, end:5, size:1, types[0]:REJECT}
+ */
+static void
+rd_kafka_share_ack_batches_collate(const rd_kafka_share_ack_batches_t *src,
+ rd_kafka_share_ack_batches_t *dst) {
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int i;
+
+ RD_LIST_FOREACH(entry, &src->entries, i) {
+ int64_t j;
+ int64_t range_start = entry->start_offset;
+ rd_kafka_share_internal_acknowledgement_type current_type =
+ rd_kafka_share_ack_convert_acquired_to_accept(
+ entry->types[0]);
+
+ /* Collate consecutive offsets with same type */
+ for (j = 1; j < entry->types_cnt; j++) {
+ rd_kafka_share_internal_acknowledgement_type this_type =
+ rd_kafka_share_ack_convert_acquired_to_accept(
+ entry->types[j]);
+
+ if (this_type != current_type) {
+ /* Type changed - emit collated entry */
+ rd_list_add(
+ &dst->entries,
+ rd_kafka_share_ack_batch_entry_collated_new(
+ range_start,
+ entry->start_offset + j - 1,
+ current_type));
+
+ /* Start new range */
+ range_start = entry->start_offset + j;
+ current_type = this_type;
+ }
+ }
+
+ /* Emit final collated entry */
+ rd_list_add(&dst->entries,
+ rd_kafka_share_ack_batch_entry_collated_new(
+ range_start, entry->end_offset, current_type));
+ }
+}
+
+/**
+ * @brief Build collated acknowledgement batches from inflight map.
+ *
+ * Iterates through the inflight acknowledgement map and collates consecutive
+ * offsets with the same type into ranges using
+ * rd_kafka_share_ack_batches_collate(). ACQUIRED type is converted to AVAILABLE
+ * (ACCEPT) for sending to broker.
+ *
+ * Each collated range is represented as rd_kafka_share_ack_batch_entry_t with
+ * types_cnt=1 and types[0] holding the single ack type for the entire range.
+ * The size field contains the actual number of offsets in the range.
+ *
+ * @param rkshare Share consumer handle
+ * @param ack_batches_out Output list to populate with
+ * rd_kafka_share_ack_batches_t*
+ */
+void rd_kafka_share_build_ack_batches_for_fetch(rd_kafka_share_t *rkshare,
+ rd_list_t *ack_batches_out) {
+ const rd_kafka_topic_partition_t *tp_key;
+ rd_kafka_share_ack_batches_t *inflight_batches;
+
+ rd_list_init(ack_batches_out, 0, rd_kafka_share_collated_batch_destroy);
+
+ /* Iterate through all topic-partitions in the inflight map */
+ RD_MAP_FOREACH(tp_key, inflight_batches,
+ &rkshare->rkshare_inflight_acks) {
+ rd_kafka_share_ack_batches_t *batch;
+
+ /* Unknown topics are filtered out during parsing */
+ rd_dassert(inflight_batches->rktpar != NULL);
+
+ /* Create output batch for this topic-partition.
+ * Reuse the rktpar from source (not copied, not destroyed). */
+ batch = rd_kafka_share_ack_batches_new();
+ batch->rktpar = inflight_batches->rktpar;
+ batch->response_leader_id =
+ inflight_batches->response_leader_id;
+ batch->response_leader_epoch =
+ inflight_batches->response_leader_epoch;
+
+ /* Collate entries from source to destination */
+ rd_kafka_share_ack_batches_collate(inflight_batches, batch);
+
+ rd_list_add(ack_batches_out, batch);
+ }
+}
+
+/**
+ * @brief Implicit acknowledgement: convert all ACQUIRED types to ACCEPT.
+ *
+ * In implicit ack mode, all records delivered to the application are
+ * automatically acknowledged as ACCEPT on the next poll(). This function
+ * walks through all entries in the inflight map and changes ACQUIRED
+ * types to ACCEPT so that rd_kafka_share_build_ack_details() will
+ * extract them for sending to the broker.
+ *
+ * @param rkshare Share consumer handle
+ */
+void rd_kafka_share_ack_all(rd_kafka_share_t *rkshare) {
+ const rd_kafka_topic_partition_t *tp_key;
+ rd_kafka_share_ack_batches_t *inflight_batches;
+
+ RD_MAP_FOREACH(tp_key, inflight_batches,
+ &rkshare->rkshare_inflight_acks) {
+ rd_kafka_share_ack_batch_entry_t *entry;
+ int i;
+ RD_LIST_FOREACH(entry, &inflight_batches->entries, i) {
+ int k;
+ for (k = 0; k < entry->types_cnt; k++) {
+ if (entry->types[k] ==
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED)
+ entry->types[k] =
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT;
+ }
+ }
+ }
+}
+
+/**
+ * @brief Free an ack details batch, its entries, and its owned rktpar.
+ *
+ * Used as rd_list_t free callback for ack_details output lists where
+ * each batch owns a copy of its rktpar.
+ */
+static void rd_kafka_share_ack_details_batch_destroy(void *ptr) {
+ rd_kafka_share_ack_batches_t *batch = ptr;
+ if (!batch)
+ return;
+ rd_kafka_share_ack_batches_destroy(batch, rd_true);
+}
+
+/**
+ * @brief Extract acknowledged (non-ACQUIRED) records from inflight map.
+ *
+ * Iterates through the inflight acknowledgement map and separates each
+ * entry's per-offset types into:
+ * - Non-ACQUIRED offsets: collated into ack_details for sending to broker
+ * (consecutive same-type offsets merged into single entry with 1 type)
+ * - ACQUIRED offsets: kept in the map as new entries (per-offset types)
+ *
+ * Map entries with no remaining ACQUIRED offsets are removed.
+ * If the map becomes empty after processing, it is cleared.
+ *
+ * @param rkshare Share consumer handle
+ * @returns Allocated list of rd_kafka_share_ack_batches_t*, or NULL if
+ * there are no ack details to send. Caller must destroy.
+ */
+rd_list_t *rd_kafka_share_build_ack_details(rd_kafka_share_t *rkshare) {
+ const rd_kafka_topic_partition_t *tp_key;
+ rd_kafka_share_ack_batches_t *inflight_batches;
+ rd_list_t keys_to_delete;
+ rd_list_t *ack_details = NULL;
+ int i;
+ rd_kafka_topic_partition_t *del_key;
+
+ rd_list_init(&keys_to_delete, 0, NULL);
+
+ rkshare->rkshare_unacked_cnt = 0;
+
+ RD_MAP_FOREACH(tp_key, inflight_batches,
+ &rkshare->rkshare_inflight_acks) {
+ rd_kafka_share_ack_batches_t *ack_batch = NULL;
+ rd_kafka_share_ack_batch_entry_t *entry;
+ rd_list_t new_entries;
+ int ei;
+
+ rd_list_init(&new_entries, 0, NULL);
+
+ RD_LIST_FOREACH(entry, &inflight_batches->entries, ei) {
+ int64_t j = 0;
+
+ while (j < entry->types_cnt) {
+ rd_kafka_share_internal_acknowledgement_type
+ run_type = entry->types[j];
+ int64_t run_start =
+ entry->start_offset + j;
+ int64_t k = j + 1;
+
+ /* Find end of consecutive same-type run */
+ while (k < entry->types_cnt &&
+ entry->types[k] == run_type)
+ k++;
+
+ int64_t run_end =
+ entry->start_offset + k - 1;
+
+ if (run_type ==
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED) {
+ /* ACQUIRED: keep in map */
+ int32_t cnt = (int32_t)(k - j);
+ rd_kafka_share_ack_batch_entry_t
+ *new_entry =
+ rd_kafka_share_ack_batch_entry_new(
+ run_start, run_end,
+ cnt);
+ int32_t m;
+ for (m = 0; m < cnt; m++)
+ new_entry->types[m] =
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED;
+ rd_list_add(&new_entries,
+ new_entry);
+ rkshare->rkshare_unacked_cnt +=
+ cnt;
+ } else {
+ /* Non-ACQUIRED: add to ack_details
+ */
+ if (!ack_batch) {
+ ack_batch =
+ rd_kafka_share_ack_batches_new();
+ ack_batch->rktpar =
+ rd_kafka_topic_partition_copy(
+ inflight_batches
+ ->rktpar);
+ ack_batch
+ ->response_leader_id =
+ inflight_batches
+ ->response_leader_id;
+ ack_batch
+ ->response_leader_epoch =
+ inflight_batches
+ ->response_leader_epoch;
+ }
+ /* Collated: 1 type for entire
+ * range */
+ rd_list_add(
+ &ack_batch->entries,
+ rd_kafka_share_ack_batch_entry_collated_new(
+ run_start, run_end,
+ run_type));
+ }
+
+ j = k;
+ }
+ }
+
+ /* Replace inflight entries with ACQUIRED-only entries */
+ RD_LIST_FOREACH(entry, &inflight_batches->entries, ei)
+ rd_kafka_share_ack_batch_entry_destroy(entry);
+ rd_list_destroy(&inflight_batches->entries);
+ inflight_batches->entries = new_entries;
+
+ /* If no ACQUIRED offsets remain, mark for removal */
+ if (rd_list_cnt(&inflight_batches->entries) == 0) {
+ rd_kafka_topic_partition_t *del_key =
+ rd_kafka_topic_partition_new_with_topic_id(
+ rd_kafka_topic_partition_get_topic_id(
+ inflight_batches->rktpar),
+ inflight_batches->rktpar->partition);
+ rd_list_add(&keys_to_delete, del_key);
+ }
+
+ if (ack_batch) {
+ if (!ack_details)
+ ack_details = rd_list_new(
+ 0,
+ rd_kafka_share_ack_details_batch_destroy);
+ rd_list_add(ack_details, ack_batch);
+ }
+ }
+
+ /* Remove map entries with no remaining ACQUIRED offsets */
+ RD_LIST_FOREACH(del_key, &keys_to_delete, i) {
+ RD_MAP_DELETE(&rkshare->rkshare_inflight_acks,
+ del_key);
+ rd_kafka_topic_partition_destroy(del_key);
+ }
+ rd_list_destroy(&keys_to_delete);
+
+ /* Clear map if empty */
+ if (RD_MAP_CNT(&rkshare->rkshare_inflight_acks) == 0)
+ RD_MAP_CLEAR(&rkshare->rkshare_inflight_acks);
+
+ return ack_details;
+}
diff --git a/src/rdkafka_share_acknowledgement.h b/src/rdkafka_share_acknowledgement.h
new file mode 100644
index 0000000000..cb2f040e5b
--- /dev/null
+++ b/src/rdkafka_share_acknowledgement.h
@@ -0,0 +1,151 @@
+/*
+ * librdkafka - The Apache Kafka C/C++ library
+ *
+ * Copyright (c) 2015-2022, Magnus Edenhill,
+ * 2023, 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.
+ */
+#ifndef _RDKAFKA_SHARE_ACKNOWLEDGEMENT_H_
+#define _RDKAFKA_SHARE_ACKNOWLEDGEMENT_H_
+
+#include "rdlist.h"
+#include "rdkafka.h"
+
+/* Forward declarations */
+typedef struct rd_kafka_share_s rd_kafka_share_t;
+typedef struct rd_kafka_op_s rd_kafka_op_t;
+
+typedef enum rd_kafka_internal_ShareAcknowledgement_type_s {
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED =
+ -1, /* Acquired records, not acknowledged yet */
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP = 0, /* gap */
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT = 1, /* accept */
+ RD_KAFKA_SHARE_INTERNAL_ACK_RELEASE = 2, /* release */
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT = 3 /* reject */
+} rd_kafka_share_internal_acknowledgement_type;
+
+/**
+ * @brief Acknowledgement batch entry for a contiguous offset range.
+ *
+ * Tracks acknowledgement status for each offset in the range.
+ * Used for building ShareAcknowledge requests.
+ *
+ * The size field always represents the number of offsets in the range
+ * (end_offset - start_offset + 1).
+ *
+ * The types_cnt field represents the actual size of the types array:
+ * - For inflight tracking: types_cnt == size (one type per offset)
+ * - For collated batches: types_cnt == 1 (single consolidated type)
+ */
+/**
+ * TODO KIP-932: Check naming.
+ */
+typedef struct rd_kafka_share_ack_batch_entry_s {
+ int64_t start_offset; /**< First offset in range */
+ int64_t end_offset; /**< Last offset in range (inclusive) */
+ int64_t size; /**< Number of offsets (end - start + 1) */
+ int32_t types_cnt; /**< Number of elements in types array */
+ int16_t delivery_count; /**< From AcquiredRecords DeliveryCount */
+ rd_kafka_share_internal_acknowledgement_type
+ *types; /**< Array of ack types */
+} rd_kafka_share_ack_batch_entry_t;
+
+/**
+ * @brief Per topic-partition inflight acknowledgement batches.
+ *
+ * Tracks all acquired records for a topic-partition that are
+ * pending acknowledgement from the application.
+ *
+ * The rktpar field contains topic, partition, and in its _private:
+ * - rktp (toppar reference, refcount held)
+ * - topic_id
+ *
+ * The response_leader_id and response_leader_epoch are the leader info
+ * at the time records were acquired. These may differ from the current
+ * leader when sending acknowledgements.
+ */
+/**
+ * TODO KIP-932: Check naming.
+ */
+typedef struct rd_kafka_share_ack_batches_s {
+ rd_kafka_topic_partition_t
+ *rktpar; /**< Topic-partition with rktp ref */
+ int32_t response_leader_id; /**< Leader broker id when records
+ * were acquired */
+ int32_t response_leader_epoch; /**< Leader epoch when records
+ * were acquired */
+ int64_t response_msgs_count; /**< Total acquired messages */
+ rd_list_t entries; /**< rd_kafka_share_ack_batch_entry_t*,
+ * sorted by start_offset */
+} rd_kafka_share_ack_batches_t;
+
+/** Allocate and initialize a share ack batch entry (offset range + types
+ * array). */
+rd_kafka_share_ack_batch_entry_t *
+rd_kafka_share_ack_batch_entry_new(int64_t start_offset,
+ int64_t end_offset,
+ int32_t types_cnt);
+/** Destroy a share ack batch entry (frees types array and entry). */
+void rd_kafka_share_ack_batch_entry_destroy(
+ rd_kafka_share_ack_batch_entry_t *entry);
+
+/** void* wrapper for rd_kafka_share_ack_batches_destroy with free_rktpar=true.
+ * Suitable as RD_MAP value destructor. */
+void rd_kafka_share_ack_batches_destroy_free(void *ptr);
+
+/** Allocate and initialize a share ack batches (list of entries). */
+rd_kafka_share_ack_batches_t *rd_kafka_share_ack_batches_new(void);
+/** Destroy share ack batches. If \p free_rktpar is true, destroys rktpar too.
+ */
+void rd_kafka_share_ack_batches_destroy(rd_kafka_share_ack_batches_t *batches,
+ rd_bool_t free_rktpar);
+
+/**
+ * @brief Transfer inflight acks from response RKO into rkshare's inflight map.
+ */
+void rd_kafka_share_build_ack_mapping(rd_kafka_share_t *rkshare,
+ rd_kafka_op_t *response_rko);
+
+/**
+ * @brief Build collated acknowledgement batches from inflight map.
+ */
+void rd_kafka_share_build_ack_batches_for_fetch(rd_kafka_share_t *rkshare,
+ rd_list_t *ack_batches_out);
+
+/**
+ * @brief Implicit ack: convert all ACQUIRED types to ACCEPT in inflight map.
+ */
+void rd_kafka_share_ack_all(rd_kafka_share_t *rkshare);
+
+/**
+ * @brief Extract acknowledged (non-ACQUIRED) records from inflight map.
+ *
+ * Non-ACQUIRED offsets are collated into ack_details for sending.
+ * ACQUIRED offsets remain in the map. Empty entries are removed.
+ *
+ * @returns Allocated list or NULL if nothing to send. Caller must destroy.
+ */
+rd_list_t *rd_kafka_share_build_ack_details(rd_kafka_share_t *rkshare);
+
+#endif /* _RDKAFKA_SHARE_ACKNOWLEDGEMENT_H_ */
diff --git a/src/rdkafka_subscription.c b/src/rdkafka_subscription.c
index 46ab544ee2..e71c7b60eb 100644
--- a/src/rdkafka_subscription.c
+++ b/src/rdkafka_subscription.c
@@ -46,6 +46,14 @@ rd_kafka_resp_err_t rd_kafka_unsubscribe(rd_kafka_t *rk) {
rd_kafka_op_req2(rkcg->rkcg_ops, RD_KAFKA_OP_SUBSCRIBE));
}
+rd_kafka_resp_err_t rd_kafka_share_unsubscribe(rd_kafka_share_t *rkshare) {
+ /**
+ * TODO KIP-932: Guard this with checks for rkshare and
+ * rkshare->rkshare_rk?
+ */
+ return rd_kafka_unsubscribe(rkshare->rkshare_rk);
+}
+
/** @returns 1 if the topic is invalid (bad regex, empty), else 0 if valid. */
static size_t _invalid_topic_cb(const rd_kafka_topic_partition_t *rktpar,
@@ -98,6 +106,16 @@ rd_kafka_subscribe(rd_kafka_t *rk,
rd_kafka_op_req(rkcg->rkcg_ops, rko, RD_POLL_INFINITE));
}
+rd_kafka_resp_err_t
+rd_kafka_share_subscribe(rd_kafka_share_t *rkshare,
+ const rd_kafka_topic_partition_list_t *topics) {
+ /**
+ * TODO KIP-932: Guard this with checks for rkshare and
+ * rkshare->rkshare_rk?
+ */
+ return rd_kafka_subscribe(rkshare->rkshare_rk, topics);
+}
+
rd_kafka_error_t *
rd_kafka_assign0(rd_kafka_t *rk,
@@ -259,6 +277,15 @@ rd_kafka_subscription(rd_kafka_t *rk,
return err;
}
+rd_kafka_resp_err_t
+rd_kafka_share_subscription(rd_kafka_share_t *rkshare,
+ rd_kafka_topic_partition_list_t **topics) {
+ /**
+ * TODO KIP-932: Guard this with checks for rkshare and
+ * rkshare->rkshare_rk?
+ */
+ return rd_kafka_subscription(rkshare->rkshare_rk, topics);
+}
rd_kafka_resp_err_t
rd_kafka_pause_partitions(rd_kafka_t *rk,
diff --git a/src/rdsysqueue.h b/src/rdsysqueue.h
index 738cdad792..4e252413f2 100644
--- a/src/rdsysqueue.h
+++ b/src/rdsysqueue.h
@@ -94,8 +94,7 @@
*/
#ifndef TAILQ_HEAD_INITIALIZER
-#define TAILQ_HEAD_INITIALIZER(head) \
- { NULL, &(head).tqh_first }
+#define TAILQ_HEAD_INITIALIZER(head) {NULL, &(head).tqh_first}
#endif
#ifndef TAILQ_INSERT_BEFORE
@@ -391,8 +390,7 @@
#endif
#ifndef SIMPLEQ_HEAD_INITIALIZER
-#define SIMPLEQ_HEAD_INITIALIZER(head) \
- { NULL, &(head).sqh_first }
+#define SIMPLEQ_HEAD_INITIALIZER(head) {NULL, &(head).sqh_first}
#endif
#ifndef SIMPLEQ_EMPTY
diff --git a/src/rdunittest.c b/src/rdunittest.c
index 547c54be54..65a10f62ec 100644
--- a/src/rdunittest.c
+++ b/src/rdunittest.c
@@ -423,6 +423,7 @@ extern int unittest_scram(void);
#endif
extern int unittest_assignors(void);
extern int unittest_map(void);
+extern int unittest_fetcher_share_filter_forward(void);
#if WITH_CURL
extern int unittest_http(void);
#endif
@@ -483,6 +484,8 @@ int rd_unittest(void) {
#endif
{"telemetry", unittest_telemetry},
{"telemetry_decode", unittest_telemetry_decode},
+ {"fetcher_share_filter_forward",
+ unittest_fetcher_share_filter_forward},
{"feature", unittest_feature},
#if WITH_SSL
{"ssl", unittest_ssl},
diff --git a/src/rdunittest_fetcher.c b/src/rdunittest_fetcher.c
new file mode 100644
index 0000000000..c209451604
--- /dev/null
+++ b/src/rdunittest_fetcher.c
@@ -0,0 +1,1133 @@
+/*
+ * librdkafka - Apache Kafka C library
+ *
+ * Copyright (c) 2012-2022, Magnus Edenhill
+ * 2023, 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.
+ */
+
+/**
+ * @brief Unit tests for Share Consumer acknowledgement flow
+ *
+ * Tests the following flow:
+ * 1. rd_kafka_share_filter_msg_from_acq_records() - Filter messages by acquired
+ * ranges
+ * 2. rd_kafka_share_build_response_rko() - Build SHARE_FETCH_RESPONSE with:
+ * - messages list (actual messages only, no GAP placeholders)
+ * - inflight_acks list (per-offset type mapping including GAPs)
+ * 3. rd_kafka_share_build_ack_mapping() - Merge inflight_acks from RKO to
+ * rkshare
+ * 4. rd_kafka_share_build_ack_batches_for_fetch() - Collate map for ShareFetch
+ * request
+ * 5. rd_kafka_q_serve_share_rkmessages() - One op per call; returns error
+ * or fills messages
+ */
+
+#include "rd.h"
+#include "rdunittest.h"
+#include "rdkafka_int.h"
+#include "rdkafka_queue.h"
+#include "rdkafka_fetcher.h"
+#include "rdkafka_partition.h"
+
+#include
+
+static rd_kafka_t *ut_create_rk(void) {
+ rd_kafka_conf_t *conf = rd_kafka_conf_new();
+ char errstr[128];
+
+ if (rd_kafka_conf_set(conf, "group.id", "ut-share-filter", errstr,
+ sizeof(errstr)) != RD_KAFKA_CONF_OK) {
+ rd_kafka_conf_destroy(conf);
+ return NULL;
+ }
+
+ rd_kafka_t *rk =
+ rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr));
+ if (!rk)
+ rd_kafka_conf_destroy(conf);
+
+ return rk;
+}
+
+static rd_kafka_toppar_t *
+ut_create_toppar(rd_kafka_t *rk, const char *topic, int32_t partition) {
+ rd_kafka_toppar_t *rktp = rd_calloc(1, sizeof(*rktp));
+ if (!rktp)
+ return NULL;
+
+ rktp->rktp_partition = partition;
+ /* Initialize refcnt to 1 - IMPORTANT for proper cleanup */
+ rd_refcnt_init(&rktp->rktp_refcnt, 1);
+
+ /* Create a minimal topic structure for testing */
+ rktp->rktp_rkt = rd_calloc(1, sizeof(*rktp->rktp_rkt));
+ if (rktp->rktp_rkt) {
+ rktp->rktp_rkt->rkt_topic = rd_kafkap_str_new(topic, -1);
+ rktp->rktp_rkt->rkt_rk = rk;
+ }
+
+ return rktp;
+}
+
+static void ut_destroy_toppar(rd_kafka_toppar_t *rktp) {
+ if (!rktp)
+ return;
+
+ /* Decrement refcnt; only free if it reaches 0.
+ * Other references (from ops) will also decrement when destroyed. */
+ if (rd_refcnt_sub(&rktp->rktp_refcnt) > 0)
+ return;
+
+ if (rktp->rktp_rkt) {
+ if (rktp->rktp_rkt->rkt_topic)
+ rd_kafkap_str_destroy(rktp->rktp_rkt->rkt_topic);
+ rd_free(rktp->rktp_rkt);
+ }
+
+ rd_free(rktp);
+}
+
+/**
+ * @brief Destroy a test op without triggering full toppar destroy.
+ *
+ * Our minimal test toppars don't have all fields initialized (queues, locks,
+ * etc.) so we can't let rd_kafka_op_destroy call rd_kafka_toppar_destroy.
+ * Instead, we manually decrement refcnt and free the op.
+ */
+static void ut_destroy_op(rd_kafka_op_t *rko) {
+ if (!rko)
+ return;
+
+ /* Decrement toppar refcnt but don't let op_destroy handle it */
+ if (rko->rko_rktp) {
+ rd_refcnt_sub(&rko->rko_rktp->rktp_refcnt);
+ rko->rko_rktp = NULL;
+ }
+
+ rd_kafka_op_destroy(rko);
+}
+
+static rd_kafka_share_t *ut_create_rkshare(rd_kafka_t *rk) {
+ rd_kafka_share_t *rkshare = rd_calloc(1, sizeof(*rkshare));
+ if (!rkshare)
+ return NULL;
+
+ rkshare->rkshare_rk = rk;
+ rkshare->rkshare_unacked_cnt = 0;
+
+ /* Inflight acks map keyed by topic_id + partition */
+ RD_MAP_INIT(&rkshare->rkshare_inflight_acks, 16,
+ rd_kafka_topic_partition_by_id_cmp,
+ rd_kafka_topic_partition_hash_by_id,
+ rd_kafka_topic_partition_destroy_free,
+ rd_kafka_share_ack_batches_destroy_free);
+
+ return rkshare;
+}
+
+static void ut_destroy_rkshare(rd_kafka_share_t *rkshare) {
+ if (!rkshare)
+ return;
+
+ /* Map value destructor handles cleanup of batches and rktpar */
+ RD_MAP_DESTROY(&rkshare->rkshare_inflight_acks);
+
+ rd_free(rkshare);
+}
+
+/**
+ * @brief Create ack batches struct with initialized rktpar.
+ *
+ * @param topic Topic name
+ * @param partition Partition number
+ * @param topic_id Topic UUID
+ * @returns Allocated batches with rktpar set up
+ */
+static rd_kafka_share_ack_batches_t *
+ut_create_batches(const char *topic,
+ int32_t partition,
+ rd_kafka_Uuid_t topic_id) {
+ rd_kafka_share_ack_batches_t *batches =
+ rd_kafka_share_ack_batches_new();
+ rd_kafka_topic_partition_private_t *parpriv;
+
+ batches->rktpar = rd_calloc(1, sizeof(*batches->rktpar));
+ batches->rktpar->topic = rd_strdup(topic);
+ batches->rktpar->partition = partition;
+ batches->rktpar->offset = RD_KAFKA_OFFSET_INVALID;
+ parpriv = rd_kafka_topic_partition_private_new();
+ parpriv->topic_id = topic_id;
+ batches->rktpar->_private = parpriv;
+
+ batches->response_leader_id = 1;
+ batches->response_leader_epoch = 1;
+
+ return batches;
+}
+
+/**
+ * @brief Create ack batch entry with types array from variadic args.
+ *
+ * @param start_offset First offset in range
+ * @param end_offset Last offset in range (inclusive)
+ * @param types_cnt Number of types (must match offset range)
+ * @param ... Variadic types (rd_kafka_share_internal_acknowledgement_type)
+ * @returns Allocated entry with types filled in
+ */
+static rd_kafka_share_ack_batch_entry_t *ut_create_entry(int64_t start_offset,
+ int64_t end_offset,
+ int32_t types_cnt,
+ ...) {
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_kafka_share_ack_batch_entry_new(start_offset, end_offset,
+ types_cnt);
+ va_list ap;
+ va_start(ap, types_cnt);
+ for (int i = 0; i < types_cnt; i++)
+ entry->types[i] =
+ (rd_kafka_share_internal_acknowledgement_type)va_arg(ap,
+ int);
+ va_end(ap);
+ return entry;
+}
+
+/**
+ * @brief Add batches to rkshare inflight map with topic_id-based key.
+ *
+ * @param rkshare Share consumer handle
+ * @param batches Batches to add (ownership transferred)
+ * @param topic_id Topic UUID for map key
+ * @param partition Partition number for map key
+ */
+static void ut_add_to_inflight(rd_kafka_share_t *rkshare,
+ rd_kafka_share_ack_batches_t *batches,
+ rd_kafka_Uuid_t topic_id,
+ int32_t partition) {
+ rd_kafka_topic_partition_t *key =
+ rd_kafka_topic_partition_new_with_topic_id(topic_id, partition);
+ RD_MAP_SET(&rkshare->rkshare_inflight_acks, key, batches);
+}
+
+/**
+ * @brief Assert collated entry has expected values.
+ *
+ * Use this macro to verify collated entries in test assertions.
+ * The entry_ptr is cast to rd_kafka_share_ack_batch_entry_t* internally.
+ */
+#define UT_ASSERT_COLLATED(entry_ptr, exp_start, exp_end, exp_type) \
+ do { \
+ rd_kafka_share_ack_batch_entry_t *_e = \
+ (rd_kafka_share_ack_batch_entry_t *)(entry_ptr); \
+ RD_UT_ASSERT( \
+ _e->start_offset == (exp_start) && \
+ _e->end_offset == (exp_end) && \
+ _e->types[0] == (exp_type), \
+ "collated entry mismatch: %" PRId64 "-%" PRId64 \
+ " type=%d, expected %" PRId64 "-%" PRId64 " type=%d", \
+ _e->start_offset, _e->end_offset, _e->types[0], \
+ (int64_t)(exp_start), (int64_t)(exp_end), (exp_type)); \
+ } while (0)
+
+static rd_kafka_op_t *ut_make_fetch_op(rd_kafka_toppar_t *rktp,
+ int64_t offset) {
+ rd_kafka_op_t *rko = rd_kafka_op_new(RD_KAFKA_OP_FETCH);
+ rko->rko_flags |= RD_KAFKA_OP_F_FREE;
+ /* Increment refcnt since op holds a reference to rktp */
+ rd_refcnt_add(&rktp->rktp_refcnt);
+ rko->rko_rktp = rktp;
+ rko->rko_u.fetch.rkm.rkm_rkmessage.partition = rktp->rktp_partition;
+ rko->rko_u.fetch.rkm.rkm_rkmessage.offset = offset;
+ rko->rko_u.fetch.rkm.rkm_rkmessage.err = RD_KAFKA_RESP_ERR_NO_ERROR;
+ return rko;
+}
+
+static rd_kafka_op_t *ut_make_error_op(rd_kafka_toppar_t *rktp,
+ int64_t offset) {
+ rd_kafka_op_t *rko = rd_kafka_op_new(RD_KAFKA_OP_CONSUMER_ERR);
+ rko->rko_flags |= RD_KAFKA_OP_F_FREE;
+ /* Increment refcnt since op holds a reference to rktp */
+ rd_refcnt_add(&rktp->rktp_refcnt);
+ rko->rko_rktp = rktp;
+ rko->rko_err = RD_KAFKA_RESP_ERR__MSG_TIMED_OUT;
+ rko->rko_u.err.offset = offset;
+ rko->rko_u.err.rkm.rkm_rkmessage.partition = rktp->rktp_partition;
+ rko->rko_u.err.rkm.rkm_rkmessage.offset = offset;
+ return rko;
+}
+
+/**
+ * Create a mock SHARE_FETCH_RESPONSE rko for testing.
+ *
+ * The new design:
+ * - messages list contains ONLY actual messages (ACQUIRED/REJECT), no GAP
+ * placeholders
+ * - inflight_acks list contains per-offset type mapping (including GAPs)
+ *
+ * @param ack_types Array describing type at each offset from acquired_start to
+ * acquired_end GAP types mean no message at that offset
+ */
+static rd_kafka_op_t *
+ut_make_share_fetch_response(rd_kafka_t *rk,
+ rd_kafka_toppar_t *rktp,
+ const char *topic,
+ int32_t partition,
+ rd_kafka_Uuid_t topic_id,
+ rd_kafka_share_internal_acknowledgement_type
+ *ack_types, /* Types for each offset */
+ int64_t acquired_start,
+ int64_t acquired_end) {
+
+ rd_kafka_op_t *response_rko =
+ rd_kafka_op_new(RD_KAFKA_OP_SHARE_FETCH_RESPONSE);
+ response_rko->rko_rk = rk;
+
+ int64_t range_size = acquired_end - acquired_start + 1;
+
+ /* Initialize lists */
+ response_rko->rko_u.share_fetch_response.message_rkos =
+ rd_list_new((int)range_size, NULL);
+ response_rko->rko_u.share_fetch_response.inflight_acks =
+ rd_list_new(1, NULL);
+
+ /* Add messages ONLY for non-GAP offsets */
+ for (int64_t i = 0; i < range_size; i++) {
+ int64_t offset = acquired_start + i;
+
+ /* Skip GAPs - they are tracked in inflight_acks only */
+ if (ack_types[i] == RD_KAFKA_SHARE_INTERNAL_ACK_GAP)
+ continue;
+
+ rd_kafka_op_t *msg_rko;
+ rd_kafka_msg_t *rkm;
+
+ if (ack_types[i] == RD_KAFKA_SHARE_INTERNAL_ACK_REJECT) {
+ msg_rko = ut_make_error_op(rktp, offset);
+ rkm = &msg_rko->rko_u.err.rkm;
+ } else {
+ msg_rko = ut_make_fetch_op(rktp, offset);
+ rkm = &msg_rko->rko_u.fetch.rkm;
+ }
+ rkm->rkm_u.consumer.ack_type = (int8_t)ack_types[i];
+ rd_list_add(
+ response_rko->rko_u.share_fetch_response.message_rkos,
+ msg_rko);
+ }
+
+ /* Build inflight_acks - this is what broker thread does */
+ rd_kafka_share_ack_batches_t *batches =
+ rd_kafka_share_ack_batches_new();
+ {
+ rd_kafka_topic_partition_private_t *parpriv;
+ batches->rktpar = rd_calloc(1, sizeof(*batches->rktpar));
+ batches->rktpar->topic = rd_strdup(topic);
+ batches->rktpar->partition = partition;
+ batches->rktpar->offset = RD_KAFKA_OFFSET_INVALID;
+ parpriv = rd_kafka_topic_partition_private_new();
+ parpriv->topic_id = topic_id;
+ batches->rktpar->_private = parpriv;
+ }
+ batches->response_leader_id = 1;
+ batches->response_leader_epoch = 1;
+ batches->response_msgs_count = (int32_t)range_size;
+
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_kafka_share_ack_batch_entry_new(acquired_start, acquired_end,
+ (int32_t)range_size);
+ memcpy(entry->types, ack_types, range_size * sizeof(*entry->types));
+
+ rd_list_add(&batches->entries, entry);
+ rd_list_add(response_rko->rko_u.share_fetch_response.inflight_acks,
+ batches);
+
+ return response_rko;
+}
+
+static void ut_destroy_share_fetch_response(rd_kafka_op_t *rko) {
+ if (!rko)
+ return;
+
+ /* Destroy message ops using ut_destroy_op to handle minimal toppars */
+ rd_kafka_op_t *msg_rko;
+ int i;
+ if (rko->rko_u.share_fetch_response.message_rkos) {
+ RD_LIST_FOREACH(
+ msg_rko, rko->rko_u.share_fetch_response.message_rkos, i) {
+ ut_destroy_op(msg_rko);
+ }
+ rd_list_destroy_free(
+ rko->rko_u.share_fetch_response.message_rkos);
+ }
+
+ /* Destroy inflight_acks */
+ rd_kafka_share_ack_batches_t *batches;
+ if (rko->rko_u.share_fetch_response.inflight_acks) {
+ RD_LIST_FOREACH(
+ batches, rko->rko_u.share_fetch_response.inflight_acks, i) {
+ rd_kafka_share_ack_batches_destroy(batches, rd_true);
+ }
+ rd_list_destroy_free(
+ rko->rko_u.share_fetch_response.inflight_acks);
+ }
+
+ rd_free(rko);
+}
+
+/*******************************************************************************
+ * Test: rd_kafka_share_filter_msg_from_acq_records()
+ * Tests filtering messages by acquired ranges
+ ******************************************************************************/
+
+static int ut_case_filter_all_in_range(rd_kafka_t *rk) {
+ rd_kafka_toppar_t *rktp = ut_create_toppar(rk, "test-topic", 0);
+ RD_UT_ASSERT(rktp != NULL, "toppar alloc failed");
+
+ rd_kafka_q_t *temp_fetchq = rd_kafka_q_new(rk);
+ rd_list_t filtered_msgs;
+ rd_list_init(&filtered_msgs, 0, NULL);
+
+ /* Add messages with offsets 0-4 */
+ for (int64_t off = 0; off <= 4; off++)
+ rd_kafka_q_enq(temp_fetchq, ut_make_fetch_op(rktp, off));
+
+ /* Acquired range: 0-4 (all messages) */
+ int64_t first[] = {0};
+ int64_t last[] = {4};
+
+ rd_kafka_share_filter_msg_from_acq_records(temp_fetchq, &filtered_msgs,
+ 1, first, last);
+
+ /* All 5 messages should be forwarded */
+ RD_UT_ASSERT(rd_list_cnt(&filtered_msgs) == 5,
+ "filtered_msgs len %d != 5", rd_list_cnt(&filtered_msgs));
+
+ /* Verify offsets and ack types */
+ for (int64_t exp = 0; exp <= 4; exp++) {
+ rd_kafka_op_t *rko = rd_list_elem(&filtered_msgs, (int)exp);
+ RD_UT_ASSERT(rko != NULL,
+ "elem returned NULL at offset %" PRId64, exp);
+ RD_UT_ASSERT(rko->rko_u.fetch.rkm.rkm_offset == exp,
+ "offset %" PRId64 " != %" PRId64,
+ rko->rko_u.fetch.rkm.rkm_offset, exp);
+ RD_UT_ASSERT(rko->rko_u.fetch.rkm.rkm_u.consumer.ack_type ==
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "ack_type %d != ACQUIRED",
+ rko->rko_u.fetch.rkm.rkm_u.consumer.ack_type);
+ ut_destroy_op(rko);
+ }
+
+ /* Note: temp_fetchq is already destroyed by
+ * rd_kafka_share_filter_msg_from_acq_records */
+ rd_list_destroy(&filtered_msgs);
+ ut_destroy_toppar(rktp);
+
+ RD_UT_SAY(" PASS: ut_case_filter_all_in_range");
+ return 0;
+}
+
+static int ut_case_filter_partial_range(rd_kafka_t *rk) {
+ rd_kafka_toppar_t *rktp = ut_create_toppar(rk, "test-topic", 0);
+ RD_UT_ASSERT(rktp != NULL, "toppar alloc failed");
+
+ rd_kafka_q_t *temp_fetchq = rd_kafka_q_new(rk);
+ rd_list_t filtered_msgs;
+ rd_list_init(&filtered_msgs, 0, NULL);
+
+ /* Add messages with offsets 0-9 */
+ for (int64_t off = 0; off <= 9; off++)
+ rd_kafka_q_enq(temp_fetchq, ut_make_fetch_op(rktp, off));
+
+ /* Acquired range: 2-5 (only some messages) */
+ int64_t first[] = {2};
+ int64_t last[] = {5};
+
+ rd_kafka_share_filter_msg_from_acq_records(temp_fetchq, &filtered_msgs,
+ 1, first, last);
+
+ /* Only 4 messages should be forwarded (offsets 2,3,4,5) */
+ RD_UT_ASSERT(rd_list_cnt(&filtered_msgs) == 4,
+ "filtered_msgs len %d != 4", rd_list_cnt(&filtered_msgs));
+
+ /* Cleanup ops in the list */
+ rd_kafka_op_t *rko;
+ int i;
+ RD_LIST_FOREACH(rko, &filtered_msgs, i) {
+ ut_destroy_op(rko);
+ }
+
+ /* Note: temp_fetchq is already destroyed by
+ * rd_kafka_share_filter_msg_from_acq_records */
+ rd_list_destroy(&filtered_msgs);
+ ut_destroy_toppar(rktp);
+
+ RD_UT_SAY(" PASS: ut_case_filter_partial_range");
+ return 0;
+}
+
+static int ut_case_filter_multiple_ranges(rd_kafka_t *rk) {
+ rd_kafka_toppar_t *rktp = ut_create_toppar(rk, "test-topic", 0);
+ RD_UT_ASSERT(rktp != NULL, "toppar alloc failed");
+
+ rd_kafka_q_t *temp_fetchq = rd_kafka_q_new(rk);
+ rd_list_t filtered_msgs;
+ rd_list_init(&filtered_msgs, 0, NULL);
+
+ /* Add messages with offsets 0-9 */
+ for (int64_t off = 0; off <= 9; off++)
+ rd_kafka_q_enq(temp_fetchq, ut_make_fetch_op(rktp, off));
+
+ /* Acquired ranges: 1-2, 5-6, 9-9 */
+ int64_t first[] = {1, 5, 9};
+ int64_t last[] = {2, 6, 9};
+
+ rd_kafka_share_filter_msg_from_acq_records(temp_fetchq, &filtered_msgs,
+ 3, first, last);
+
+ /* 5 messages should be forwarded (1,2,5,6,9) */
+ RD_UT_ASSERT(rd_list_cnt(&filtered_msgs) == 5,
+ "filtered_msgs len %d != 5", rd_list_cnt(&filtered_msgs));
+
+ /* Cleanup ops in the list */
+ rd_kafka_op_t *rko;
+ int i;
+ RD_LIST_FOREACH(rko, &filtered_msgs, i) {
+ ut_destroy_op(rko);
+ }
+
+ /* Note: temp_fetchq is already destroyed by
+ * rd_kafka_share_filter_msg_from_acq_records */
+ rd_list_destroy(&filtered_msgs);
+ ut_destroy_toppar(rktp);
+
+ RD_UT_SAY(" PASS: ut_case_filter_multiple_ranges");
+ return 0;
+}
+
+/*******************************************************************************
+ * Test: RKO Structure Creation (simulating broker thread)
+ * Tests that SHARE_FETCH_RESPONSE RKO is built correctly
+ ******************************************************************************/
+
+static int ut_case_rko_structure_all_acquired(rd_kafka_t *rk) {
+ rd_kafka_toppar_t *rktp = ut_create_toppar(rk, "T1", 0);
+ RD_UT_ASSERT(rktp != NULL, "toppar alloc failed");
+
+ /* Create response with all ACQUIRED messages */
+ rd_kafka_share_internal_acknowledgement_type types[] = {
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED};
+
+ rd_kafka_op_t *response_rko = ut_make_share_fetch_response(
+ rk, rktp, "T1", 0, RD_KAFKA_UUID_ZERO, types, 1, 6);
+
+ /* Verify the response structure */
+ /* All 6 messages should be present (no GAPs) */
+ RD_UT_ASSERT(
+ rd_list_cnt(
+ response_rko->rko_u.share_fetch_response.message_rkos) == 6,
+ "message cnt %d != 6",
+ rd_list_cnt(response_rko->rko_u.share_fetch_response.message_rkos));
+
+ /* Verify inflight_acks was created */
+ RD_UT_ASSERT(
+ rd_list_cnt(
+ response_rko->rko_u.share_fetch_response.inflight_acks) == 1,
+ "inflight_acks cnt %d != 1",
+ rd_list_cnt(
+ response_rko->rko_u.share_fetch_response.inflight_acks));
+
+ ut_destroy_share_fetch_response(response_rko);
+ ut_destroy_toppar(rktp);
+
+ RD_UT_SAY(" PASS: ut_case_rko_structure_all_acquired");
+ return 0;
+}
+
+static int ut_case_rko_structure_with_gaps(rd_kafka_t *rk) {
+ rd_kafka_toppar_t *rktp = ut_create_toppar(rk, "T1", 0);
+ RD_UT_ASSERT(rktp != NULL, "toppar alloc failed");
+
+ /* Create response with gaps: ACQ, ACQ, GAP, GAP, ACQ, ACQ */
+ rd_kafka_share_internal_acknowledgement_type types[] = {
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED};
+
+ rd_kafka_op_t *response_rko = ut_make_share_fetch_response(
+ rk, rktp, "T1", 0, RD_KAFKA_UUID_ZERO, types, 1, 6);
+
+ /* Only 4 messages should be present (2 GAPs excluded) */
+ RD_UT_ASSERT(
+ rd_list_cnt(
+ response_rko->rko_u.share_fetch_response.message_rkos) == 4,
+ "message cnt %d != 4",
+ rd_list_cnt(response_rko->rko_u.share_fetch_response.message_rkos));
+
+ /* Verify inflight_acks has all 6 offsets including GAPs */
+ rd_kafka_share_ack_batches_t *batches = rd_list_elem(
+ response_rko->rko_u.share_fetch_response.inflight_acks, 0);
+ RD_UT_ASSERT(batches != NULL, "inflight_acks[0] is NULL");
+
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_list_elem(&batches->entries, 0);
+ RD_UT_ASSERT(entry != NULL, "entries[0] is NULL");
+ RD_UT_ASSERT(entry->size == 6, "entry size %" PRId64 " != 6",
+ entry->size);
+
+ /* Verify types array has GAPs at positions 2,3 */
+ RD_UT_ASSERT(entry->types[0] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[0] != ACQ");
+ RD_UT_ASSERT(entry->types[1] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[1] != ACQ");
+ RD_UT_ASSERT(entry->types[2] == RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ "type[2] != GAP");
+ RD_UT_ASSERT(entry->types[3] == RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ "type[3] != GAP");
+ RD_UT_ASSERT(entry->types[4] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[4] != ACQ");
+ RD_UT_ASSERT(entry->types[5] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[5] != ACQ");
+
+ ut_destroy_share_fetch_response(response_rko);
+ ut_destroy_toppar(rktp);
+
+ RD_UT_SAY(" PASS: ut_case_rko_structure_with_gaps");
+ return 0;
+}
+
+static int ut_case_rko_structure_with_rejects(rd_kafka_t *rk) {
+ rd_kafka_toppar_t *rktp = ut_create_toppar(rk, "T1", 0);
+ RD_UT_ASSERT(rktp != NULL, "toppar alloc failed");
+
+ /* Create response with rejects: ACQ, REJ, GAP, REJ, REJ, ACQ */
+ rd_kafka_share_internal_acknowledgement_type types[] = {
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED};
+
+ rd_kafka_op_t *response_rko = ut_make_share_fetch_response(
+ rk, rktp, "T1", 0, RD_KAFKA_UUID_ZERO, types, 1, 6);
+
+ /* 5 messages present (1 GAP excluded): ACQ, REJ, REJ, REJ, ACQ */
+ RD_UT_ASSERT(
+ rd_list_cnt(
+ response_rko->rko_u.share_fetch_response.message_rkos) == 5,
+ "message cnt %d != 5",
+ rd_list_cnt(response_rko->rko_u.share_fetch_response.message_rkos));
+
+ /* Count CONSUMER_ERR ops (should be 3 REJECTs) */
+ rd_kafka_op_t *msg_rko;
+ int i, reject_cnt = 0;
+ RD_LIST_FOREACH(
+ msg_rko, response_rko->rko_u.share_fetch_response.message_rkos, i) {
+ if (msg_rko->rko_type == RD_KAFKA_OP_CONSUMER_ERR)
+ reject_cnt++;
+ }
+ RD_UT_ASSERT(reject_cnt == 3, "reject cnt %d != 3", reject_cnt);
+
+ /* Verify inflight_acks has correct types */
+ rd_kafka_share_ack_batches_t *batches = rd_list_elem(
+ response_rko->rko_u.share_fetch_response.inflight_acks, 0);
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_list_elem(&batches->entries, 0);
+
+ RD_UT_ASSERT(entry->types[0] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[0] != ACQ");
+ RD_UT_ASSERT(entry->types[1] == RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ "type[1] != REJ");
+ RD_UT_ASSERT(entry->types[2] == RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ "type[2] != GAP");
+ RD_UT_ASSERT(entry->types[3] == RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ "type[3] != REJ");
+ RD_UT_ASSERT(entry->types[4] == RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ "type[4] != REJ");
+ RD_UT_ASSERT(entry->types[5] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[5] != ACQ");
+
+ ut_destroy_share_fetch_response(response_rko);
+ ut_destroy_toppar(rktp);
+
+ RD_UT_SAY(" PASS: ut_case_rko_structure_with_rejects");
+ return 0;
+}
+
+/*******************************************************************************
+ * Test: rd_kafka_share_build_ack_mapping()
+ * Tests merging inflight_acks from RKO to rkshare map
+ ******************************************************************************/
+
+static int ut_case_merge_single_partition(rd_kafka_t *rk) {
+ rd_kafka_share_t *rkshare = ut_create_rkshare(rk);
+ RD_UT_ASSERT(rkshare != NULL, "rkshare alloc failed");
+
+ rd_kafka_toppar_t *rktp = ut_create_toppar(rk, "T1", 0);
+ RD_UT_ASSERT(rktp != NULL, "toppar alloc failed");
+
+ /* Create response with: ACQ, ACQ, GAP, ACQ, ACQ, REJ */
+ rd_kafka_share_internal_acknowledgement_type types[] = {
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT};
+
+ rd_kafka_op_t *response_rko = ut_make_share_fetch_response(
+ rk, rktp, "T1", 0, RD_KAFKA_UUID_ZERO, types, 1, 6);
+
+ /* Verify rkshare map is empty before merge */
+ RD_UT_ASSERT(RD_MAP_CNT(&rkshare->rkshare_inflight_acks) == 0,
+ "rkshare map not empty before merge");
+
+ /* Call the merge function (simulating app thread) */
+ rd_kafka_share_build_ack_mapping(rkshare, response_rko);
+
+ /* Verify rkshare map has 1 partition */
+ RD_UT_ASSERT(RD_MAP_CNT(&rkshare->rkshare_inflight_acks) == 1,
+ "rkshare map cnt %d != 1",
+ (int)RD_MAP_CNT(&rkshare->rkshare_inflight_acks));
+
+ /* Verify unacked count (only ACQUIRED types: 4 out of 6) */
+ RD_UT_ASSERT(rkshare->rkshare_unacked_cnt == 4,
+ "unacked cnt %" PRId64 " != 4",
+ rkshare->rkshare_unacked_cnt);
+
+ /* Lookup the merged batches (map keyed by topic_id + partition) */
+ rd_kafka_topic_partition_t *lookup_key =
+ rd_kafka_topic_partition_new_with_topic_id(RD_KAFKA_UUID_ZERO, 0);
+ rd_kafka_share_ack_batches_t *merged =
+ RD_MAP_GET(&rkshare->rkshare_inflight_acks, lookup_key);
+ rd_kafka_topic_partition_destroy_free(lookup_key);
+ RD_UT_ASSERT(merged != NULL, "merged batches not found");
+
+ /* Verify merged data */
+ RD_UT_ASSERT(merged->rktpar != NULL, "rktpar is NULL");
+ RD_UT_ASSERT(strcmp(merged->rktpar->topic, "T1") == 0,
+ "topic mismatch");
+ RD_UT_ASSERT(merged->rktpar->partition == 0, "partition mismatch");
+ RD_UT_ASSERT(rd_list_cnt(&merged->entries) == 1,
+ "entries cnt mismatch");
+
+ rd_kafka_share_ack_batch_entry_t *entry =
+ rd_list_elem(&merged->entries, 0);
+ RD_UT_ASSERT(entry->start_offset == 1 && entry->end_offset == 6,
+ "offset range mismatch");
+
+ /* Verify types were copied correctly */
+ RD_UT_ASSERT(entry->types[0] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[0]");
+ RD_UT_ASSERT(entry->types[1] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[1]");
+ RD_UT_ASSERT(entry->types[2] == RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ "type[2]");
+ RD_UT_ASSERT(entry->types[3] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[3]");
+ RD_UT_ASSERT(entry->types[4] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "type[4]");
+ RD_UT_ASSERT(entry->types[5] == RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ "type[5]");
+
+ ut_destroy_share_fetch_response(response_rko);
+ ut_destroy_toppar(rktp);
+ ut_destroy_rkshare(rkshare);
+
+ RD_UT_SAY(" PASS: ut_case_merge_single_partition");
+ return 0;
+}
+
+static int ut_case_merge_multiple_rkos(rd_kafka_t *rk) {
+ rd_kafka_share_t *rkshare = ut_create_rkshare(rk);
+ RD_UT_ASSERT(rkshare != NULL, "rkshare alloc failed");
+
+ rd_kafka_toppar_t *rktp1 = ut_create_toppar(rk, "T1", 0);
+ rd_kafka_toppar_t *rktp2 = ut_create_toppar(rk, "T2", 0);
+ RD_UT_ASSERT(rktp1 != NULL && rktp2 != NULL, "toppar alloc failed");
+
+ /* First RKO from broker 1: T1-0 with ACQ, ACQ, GAP */
+ rd_kafka_share_internal_acknowledgement_type types1[] = {
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP};
+ rd_kafka_op_t *rko1 = ut_make_share_fetch_response(
+ rk, rktp1, "T1", 0, RD_KAFKA_UUID_ZERO, types1, 1, 3);
+
+ /* Second RKO from broker 2: T2-0 with ACQ, REJ, ACQ (distinct topic_id)
+ */
+ static const rd_kafka_Uuid_t ut_topic_id_t2 = {0, 1, ""};
+ rd_kafka_share_internal_acknowledgement_type types2[] = {
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED};
+ rd_kafka_op_t *rko2 = ut_make_share_fetch_response(
+ rk, rktp2, "T2", 0, ut_topic_id_t2, types2, 10, 12);
+
+ /* Merge first RKO */
+ rd_kafka_share_build_ack_mapping(rkshare, rko1);
+
+ /* Verify after first merge (2 ACQUIRED out of 3: ACQ, ACQ, GAP) */
+ RD_UT_ASSERT(RD_MAP_CNT(&rkshare->rkshare_inflight_acks) == 1,
+ "map cnt %d != 1 after first merge",
+ (int)RD_MAP_CNT(&rkshare->rkshare_inflight_acks));
+ RD_UT_ASSERT(rkshare->rkshare_unacked_cnt == 2,
+ "unacked %" PRId64 " != 2", rkshare->rkshare_unacked_cnt);
+
+ /* Merge second RKO */
+ rd_kafka_share_build_ack_mapping(rkshare, rko2);
+
+ /* Verify after second merge (2 + 2 ACQUIRED: ACQ, REJ, ACQ) */
+ RD_UT_ASSERT(RD_MAP_CNT(&rkshare->rkshare_inflight_acks) == 2,
+ "map cnt %d != 2 after second merge",
+ (int)RD_MAP_CNT(&rkshare->rkshare_inflight_acks));
+ RD_UT_ASSERT(rkshare->rkshare_unacked_cnt == 4,
+ "unacked %" PRId64 " != 4", rkshare->rkshare_unacked_cnt);
+
+ /* Verify T1-0 (map keyed by topic_id + partition) */
+ rd_kafka_topic_partition_t *key1 =
+ rd_kafka_topic_partition_new_with_topic_id(RD_KAFKA_UUID_ZERO, 0);
+ rd_kafka_share_ack_batches_t *batches1 =
+ RD_MAP_GET(&rkshare->rkshare_inflight_acks, key1);
+ rd_kafka_topic_partition_destroy_free(key1);
+ RD_UT_ASSERT(batches1 != NULL, "T1-0 not found");
+ RD_UT_ASSERT(rd_list_cnt(&batches1->entries) == 1, "T1-0 entries cnt");
+
+ /* Verify T2-0 (different topic_id) */
+ rd_kafka_topic_partition_t *key2 =
+ rd_kafka_topic_partition_new_with_topic_id(ut_topic_id_t2, 0);
+ rd_kafka_share_ack_batches_t *batches2 =
+ RD_MAP_GET(&rkshare->rkshare_inflight_acks, key2);
+ rd_kafka_topic_partition_destroy_free(key2);
+ RD_UT_ASSERT(batches2 != NULL, "T2-0 not found");
+ RD_UT_ASSERT(rd_list_cnt(&batches2->entries) == 1, "T2-0 entries cnt");
+
+ rd_kafka_share_ack_batch_entry_t *entry2 =
+ rd_list_elem(&batches2->entries, 0);
+ RD_UT_ASSERT(entry2->start_offset == 10 && entry2->end_offset == 12,
+ "T2-0 offset range mismatch");
+ RD_UT_ASSERT(entry2->types[0] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "T2 type[0]");
+ RD_UT_ASSERT(entry2->types[1] == RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ "T2 type[1]");
+ RD_UT_ASSERT(entry2->types[2] == RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ "T2 type[2]");
+
+ ut_destroy_share_fetch_response(rko1);
+ ut_destroy_share_fetch_response(rko2);
+ ut_destroy_toppar(rktp1);
+ ut_destroy_toppar(rktp2);
+ ut_destroy_rkshare(rkshare);
+
+ RD_UT_SAY(" PASS: ut_case_merge_multiple_rkos");
+ return 0;
+}
+
+/*******************************************************************************
+ * Test: rd_kafka_share_build_ack_batches_for_fetch()
+ * Tests collating inflight map into ranges for ShareFetch
+ ******************************************************************************/
+
+static int ut_case_collate_all_same_type(rd_kafka_t *rk) {
+ rd_kafka_share_t *rkshare = ut_create_rkshare(rk);
+ RD_UT_ASSERT(rkshare != NULL, "rkshare alloc failed");
+
+ /* Populate inflight map with all ACQUIRED */
+ rd_kafka_share_ack_batches_t *batches =
+ ut_create_batches("T1", 0, RD_KAFKA_UUID_ZERO);
+
+ rd_kafka_share_ack_batch_entry_t *entry =
+ ut_create_entry(1, 4, 4, RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED);
+ rd_list_add(&batches->entries, entry);
+
+ ut_add_to_inflight(rkshare, batches, RD_KAFKA_UUID_ZERO, 0);
+
+ /* Call collate function */
+ rd_list_t ack_batches_out;
+ rd_kafka_share_build_ack_batches_for_fetch(rkshare, &ack_batches_out);
+
+ /* Should produce 1 batch with 1 entry (all ACCEPT) */
+ RD_UT_ASSERT(rd_list_cnt(&ack_batches_out) == 1,
+ "ack_batches cnt %d != 1", rd_list_cnt(&ack_batches_out));
+
+ rd_kafka_share_ack_batches_t *out_batch =
+ rd_list_elem(&ack_batches_out, 0);
+ RD_UT_ASSERT(rd_list_cnt(&out_batch->entries) == 1,
+ "entries cnt %d != 1", rd_list_cnt(&out_batch->entries));
+
+ rd_kafka_share_ack_batch_entry_t *collated =
+ rd_list_elem(&out_batch->entries, 0);
+ UT_ASSERT_COLLATED(collated, 1, 4, RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+ RD_UT_ASSERT(collated->size == 4, "size %" PRId64 " != 4",
+ collated->size);
+ RD_UT_ASSERT(collated->types_cnt == 1, "types_cnt %d != 1",
+ collated->types_cnt);
+
+ rd_list_destroy(&ack_batches_out);
+ ut_destroy_rkshare(rkshare);
+
+ RD_UT_SAY(" PASS: ut_case_collate_all_same_type");
+ return 0;
+}
+
+static int ut_case_collate_mixed_types(rd_kafka_t *rk) {
+ rd_kafka_share_t *rkshare = ut_create_rkshare(rk);
+ RD_UT_ASSERT(rkshare != NULL, "rkshare alloc failed");
+
+ /* Populate: ACQ, ACQ, GAP, ACQ, ACQ, REJ, REJ, ACQ, ACQ */
+ rd_kafka_share_ack_batches_t *batches =
+ ut_create_batches("T1", 0, RD_KAFKA_UUID_ZERO);
+
+ rd_kafka_share_ack_batch_entry_t *entry =
+ ut_create_entry(1, 9, 9, RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED);
+ rd_list_add(&batches->entries, entry);
+
+ ut_add_to_inflight(rkshare, batches, RD_KAFKA_UUID_ZERO, 0);
+
+ /* Call collate function */
+ rd_list_t ack_batches_out;
+ rd_kafka_share_build_ack_batches_for_fetch(rkshare, &ack_batches_out);
+
+ /* Should produce 1 batch with 5 entries */
+ RD_UT_ASSERT(rd_list_cnt(&ack_batches_out) == 1,
+ "ack_batches cnt %d != 1", rd_list_cnt(&ack_batches_out));
+
+ rd_kafka_share_ack_batches_t *out_batch =
+ rd_list_elem(&ack_batches_out, 0);
+ RD_UT_ASSERT(rd_list_cnt(&out_batch->entries) == 5,
+ "entries cnt %d != 5", rd_list_cnt(&out_batch->entries));
+
+ /* Verify entries: {1-2, ACCEPT}, {3-3, GAP}, {4-5, ACCEPT},
+ * {6-7, REJECT}, {8-9, ACCEPT} */
+ UT_ASSERT_COLLATED(rd_list_elem(&out_batch->entries, 0), 1, 2,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+ UT_ASSERT_COLLATED(rd_list_elem(&out_batch->entries, 1), 3, 3,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP);
+ UT_ASSERT_COLLATED(rd_list_elem(&out_batch->entries, 2), 4, 5,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+ UT_ASSERT_COLLATED(rd_list_elem(&out_batch->entries, 3), 6, 7,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT);
+ UT_ASSERT_COLLATED(rd_list_elem(&out_batch->entries, 4), 8, 9,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+
+ rd_list_destroy(&ack_batches_out);
+ ut_destroy_rkshare(rkshare);
+
+ RD_UT_SAY(" PASS: ut_case_collate_mixed_types");
+ return 0;
+}
+
+static int ut_case_collate_empty_map(rd_kafka_t *rk) {
+ rd_kafka_share_t *rkshare = ut_create_rkshare(rk);
+ RD_UT_ASSERT(rkshare != NULL, "rkshare alloc failed");
+
+ /* Don't add anything to inflight map */
+
+ /* Call collate function */
+ rd_list_t ack_batches_out;
+ rd_kafka_share_build_ack_batches_for_fetch(rkshare, &ack_batches_out);
+
+ /* Should produce empty list */
+ RD_UT_ASSERT(rd_list_cnt(&ack_batches_out) == 0,
+ "ack_batches cnt %d != 0", rd_list_cnt(&ack_batches_out));
+
+ rd_list_destroy(&ack_batches_out);
+ ut_destroy_rkshare(rkshare);
+
+ RD_UT_SAY(" PASS: ut_case_collate_empty_map");
+ return 0;
+}
+
+/*******************************************************************************
+ * Test: Full flow with multiple partitions
+ * Tests the complete scenario from user's example
+ ******************************************************************************/
+
+static int ut_case_full_flow_multi_partition(rd_kafka_t *rk) {
+ rd_kafka_share_t *rkshare = ut_create_rkshare(rk);
+ RD_UT_ASSERT(rkshare != NULL, "rkshare alloc failed");
+
+ /*
+ * Test case:
+ * Tp1: 1:ACQ, 2:ACQ, 3:GAP, 4:GAP, 5:ACQ, 6:ACQ
+ * Tp2: 1:ACQ, 2:REJ, 3:GAP, 4:REJ, 5:REJ, 6:ACQ, 7:ACQ, 8:GAP
+ */
+
+ /* Populate Tp1 */
+ rd_kafka_share_ack_batches_t *batches1 =
+ ut_create_batches("Tp1", 0, RD_KAFKA_UUID_ZERO);
+ rd_kafka_share_ack_batch_entry_t *entry1 = ut_create_entry(
+ 1, 6, 6, RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP, RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED);
+ rd_list_add(&batches1->entries, entry1);
+ ut_add_to_inflight(rkshare, batches1, RD_KAFKA_UUID_ZERO, 0);
+
+ /* Populate Tp2 (distinct topic_id) */
+ static const rd_kafka_Uuid_t ut_topic_id_tp2 = {0, 2, ""};
+ rd_kafka_share_ack_batches_t *batches2 =
+ ut_create_batches("Tp2", 0, ut_topic_id_tp2);
+ batches2->response_leader_id = 2; /* Override default */
+ rd_kafka_share_ack_batch_entry_t *entry2 = ut_create_entry(
+ 1, 8, 8, RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT, RD_KAFKA_SHARE_INTERNAL_ACK_GAP,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACQUIRED,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP);
+ rd_list_add(&batches2->entries, entry2);
+ ut_add_to_inflight(rkshare, batches2, ut_topic_id_tp2, 0);
+
+ /* Call collate function */
+ rd_list_t ack_batches_out;
+ rd_kafka_share_build_ack_batches_for_fetch(rkshare, &ack_batches_out);
+
+ /* Should produce 2 batches */
+ RD_UT_ASSERT(rd_list_cnt(&ack_batches_out) == 2,
+ "ack_batches cnt %d != 2", rd_list_cnt(&ack_batches_out));
+
+ /* Find Tp1 and Tp2 batches */
+ rd_kafka_share_ack_batches_t *tp1_batch = NULL;
+ rd_kafka_share_ack_batches_t *tp2_batch = NULL;
+ rd_kafka_share_ack_batches_t *batch;
+ int i;
+
+ RD_LIST_FOREACH(batch, &ack_batches_out, i) {
+ if (batch->rktpar && strcmp(batch->rktpar->topic, "Tp1") == 0)
+ tp1_batch = batch;
+ else if (batch->rktpar &&
+ strcmp(batch->rktpar->topic, "Tp2") == 0)
+ tp2_batch = batch;
+ }
+
+ RD_UT_ASSERT(tp1_batch != NULL, "Tp1 batch not found");
+ RD_UT_ASSERT(tp2_batch != NULL, "Tp2 batch not found");
+
+ /* Tp1 expected: {1-2, ACCEPT}, {3-4, GAP}, {5-6, ACCEPT} */
+ RD_UT_ASSERT(rd_list_cnt(&tp1_batch->entries) == 3,
+ "Tp1 entries cnt %d != 3",
+ rd_list_cnt(&tp1_batch->entries));
+
+ UT_ASSERT_COLLATED(rd_list_elem(&tp1_batch->entries, 0), 1, 2,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+ UT_ASSERT_COLLATED(rd_list_elem(&tp1_batch->entries, 1), 3, 4,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP);
+ UT_ASSERT_COLLATED(rd_list_elem(&tp1_batch->entries, 2), 5, 6,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+
+ /* Tp2 expected: {1-1, ACCEPT}, {2-2, REJECT}, {3-3, GAP},
+ * {4-5, REJECT}, {6-7, ACCEPT}, {8-8, GAP} */
+ RD_UT_ASSERT(rd_list_cnt(&tp2_batch->entries) == 6,
+ "Tp2 entries cnt %d != 6",
+ rd_list_cnt(&tp2_batch->entries));
+
+ UT_ASSERT_COLLATED(rd_list_elem(&tp2_batch->entries, 0), 1, 1,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+ UT_ASSERT_COLLATED(rd_list_elem(&tp2_batch->entries, 1), 2, 2,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT);
+ UT_ASSERT_COLLATED(rd_list_elem(&tp2_batch->entries, 2), 3, 3,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP);
+ UT_ASSERT_COLLATED(rd_list_elem(&tp2_batch->entries, 3), 4, 5,
+ RD_KAFKA_SHARE_INTERNAL_ACK_REJECT);
+ UT_ASSERT_COLLATED(rd_list_elem(&tp2_batch->entries, 4), 6, 7,
+ RD_KAFKA_SHARE_INTERNAL_ACK_ACCEPT);
+ UT_ASSERT_COLLATED(rd_list_elem(&tp2_batch->entries, 5), 8, 8,
+ RD_KAFKA_SHARE_INTERNAL_ACK_GAP);
+
+ rd_list_destroy(&ack_batches_out);
+ ut_destroy_rkshare(rkshare);
+
+ RD_UT_SAY(" PASS: ut_case_full_flow_multi_partition");
+ return 0;
+}
+
+/*******************************************************************************
+ * Main Test Entry Point
+ ******************************************************************************/
+
+int unittest_fetcher_share_filter_forward(void) {
+ rd_kafka_t *rk = ut_create_rk();
+ RD_UT_ASSERT(rk != NULL, "rd_kafka_new failed");
+
+ RD_UT_SAY("Testing rd_kafka_share_filter_msg_from_acq_records()...");
+ if (ut_case_filter_all_in_range(rk) ||
+ ut_case_filter_partial_range(rk) ||
+ ut_case_filter_multiple_ranges(rk)) {
+ rd_kafka_destroy(rk);
+ return 1;
+ }
+
+ RD_UT_SAY(
+ "Testing RKO structure creation (broker thread simulation)...");
+ if (ut_case_rko_structure_all_acquired(rk) ||
+ ut_case_rko_structure_with_gaps(rk) ||
+ ut_case_rko_structure_with_rejects(rk)) {
+ rd_kafka_destroy(rk);
+ return 1;
+ }
+
+ RD_UT_SAY("Testing rd_kafka_share_build_ack_mapping() merge...");
+ if (ut_case_merge_single_partition(rk) ||
+ ut_case_merge_multiple_rkos(rk)) {
+ rd_kafka_destroy(rk);
+ return 1;
+ }
+
+ RD_UT_SAY("Testing rd_kafka_share_build_ack_batches_for_fetch()...");
+ if (ut_case_collate_all_same_type(rk) ||
+ ut_case_collate_mixed_types(rk) || ut_case_collate_empty_map(rk)) {
+ rd_kafka_destroy(rk);
+ return 1;
+ }
+
+ RD_UT_SAY("Testing full flow with multi-partition...");
+ if (ut_case_full_flow_multi_partition(rk)) {
+ rd_kafka_destroy(rk);
+ return 1;
+ }
+
+ rd_kafka_destroy(rk);
+ RD_UT_PASS();
+}
diff --git a/src/rdwin32.h b/src/rdwin32.h
index 40ea43a7ac..24d88f40dc 100644
--- a/src/rdwin32.h
+++ b/src/rdwin32.h
@@ -231,8 +231,7 @@ static RD_INLINE RD_UNUSED const char *rd_getenv(const char *env,
/**
* Empty struct initializer
*/
-#define RD_ZERO_INIT \
- { 0 }
+#define RD_ZERO_INIT {0}
#ifndef __cplusplus
/**
diff --git a/tests/0155-share_group_heartbeat_mock.c b/tests/0155-share_group_heartbeat_mock.c
new file mode 100644
index 0000000000..900c683c0d
--- /dev/null
+++ b/tests/0155-share_group_heartbeat_mock.c
@@ -0,0 +1,2466 @@
+#include "test.h"
+
+#include "../src/rdkafka_proto.h"
+
+/**
+ * @name Mock tests for share consumer and ShareGroupHeartbeat
+ */
+
+static rd_bool_t is_share_heartbeat_request(rd_kafka_mock_request_t *request,
+ void *opaque) {
+ return rd_kafka_mock_request_api_key(request) ==
+ RD_KAFKAP_ShareGroupHeartbeat;
+}
+
+/**
+ * @brief Wait for at least \p num ShareGroupHeartbeat requests
+ * to be received by the mock cluster.
+ *
+ * @return Number of heartbeats received.
+ */
+static int wait_share_heartbeats(rd_kafka_mock_cluster_t *mcluster,
+ int num,
+ int confidence_interval) {
+ return test_mock_wait_matching_requests(
+ mcluster, num, confidence_interval, is_share_heartbeat_request,
+ NULL);
+}
+
+/**
+ * @brief Create a share consumer connected to mock cluster.
+ */
+static rd_kafka_t *create_share_consumer(const char *bootstraps,
+ const char *group_id) {
+ rd_kafka_conf_t *conf;
+ rd_kafka_t *rk;
+ char errstr[512];
+
+ test_conf_init(&conf, NULL, 0);
+ test_conf_set(conf, "bootstrap.servers", bootstraps);
+ test_conf_set(conf, "group.id", group_id);
+ test_conf_set(conf, "share.consumer", "true");
+ test_conf_set(conf, "group.protocol", "consumer");
+ test_conf_set(conf, "auto.offset.reset", "earliest");
+
+ rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr));
+ TEST_ASSERT(rk != NULL, "Failed to create share consumer: %s", errstr);
+
+ return rk;
+}
+
+/**
+ * @brief Test basic ShareGroupHeartbeat flow:
+ * join, receive assignment, heartbeats, leave.
+ */
+static void do_test_share_group_heartbeat_basic(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_t *c;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for join heartbeat */
+ found_heartbeats = wait_share_heartbeats(mcluster, 1, 500);
+ TEST_ASSERT(found_heartbeats >= 1,
+ "Expected at least 1 heartbeat, got %d", found_heartbeats);
+
+ /* Poll to process response and trigger more heartbeats */
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify assignment received (matches testReconcileNewPartitions) */
+ {
+ rd_kafka_topic_partition_list_t *assignment;
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions assigned, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+ }
+
+ /* Verify multiple heartbeats */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 200);
+ TEST_ASSERT(found_heartbeats >= 2,
+ "Expected at least 2 heartbeats, got %d", found_heartbeats);
+
+ /* Close consumer (sends leave heartbeat) */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ /* Verify leave heartbeat was sent */
+ found_heartbeats = wait_share_heartbeats(mcluster, 3, 200);
+
+ /* Verify no more heartbeats after leave */
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ rd_kafka_mock_start_request_tracking(mcluster);
+ rd_sleep(3);
+ found_heartbeats = wait_share_heartbeats(mcluster, 0, 100);
+ TEST_ASSERT(found_heartbeats == 0,
+ "Expected 0 heartbeats after leave, got %d",
+ found_heartbeats);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Test assignment redistribution when consumers join/leave.
+ */
+static void do_test_share_group_assignment_rebalance(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_topic_partition_list_t *c1_assignment, *c2_assignment;
+ rd_kafka_t *c1, *c2;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-rebalance";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c1 = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, subscription));
+
+ /* C1 joins - should get all 3 partitions */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assignment));
+ TEST_ASSERT(c1_assignment->cnt == 3,
+ "Expected C1 to have 3 partitions, got %d",
+ c1_assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(c1_assignment);
+
+ /* C2 joins - partitions should be redistributed */
+ c2 = create_share_consumer(bootstraps, group);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ wait_share_heartbeats(mcluster, 3, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+ rd_kafka_consumer_poll(c2, 2000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assignment));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assignment));
+ TEST_ASSERT(c1_assignment->cnt + c2_assignment->cnt == 3,
+ "Expected total 3 partitions, got %d + %d = %d",
+ c1_assignment->cnt, c2_assignment->cnt,
+ c1_assignment->cnt + c2_assignment->cnt);
+ TEST_ASSERT(c1_assignment->cnt > 0 && c2_assignment->cnt > 0,
+ "Expected both consumers to have partitions, "
+ "got C1=%d, C2=%d",
+ c1_assignment->cnt, c2_assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(c1_assignment);
+ rd_kafka_topic_partition_list_destroy(c2_assignment);
+
+ /* C2 leaves - C1 should get all partitions back */
+ rd_kafka_consumer_close(c2);
+ rd_kafka_destroy(c2);
+
+ rd_kafka_consumer_poll(c1, 6000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assignment));
+ TEST_ASSERT(c1_assignment->cnt == 3,
+ "Expected C1 to have 3 partitions after C2 left, got %d",
+ c1_assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(c1_assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c1);
+ rd_kafka_destroy(c1);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+static int count_topic_partitions(rd_kafka_topic_partition_list_t *assignment,
+ const char *topic) {
+ int i, count = 0;
+ for (i = 0; i < assignment->cnt; i++) {
+ if (strcmp(assignment->elems[i].topic, topic) == 0)
+ count++;
+ }
+ return count;
+}
+
+/**
+ * @brief Test multi-topic assignment with mixed subscriptions.
+ * C1: both topics, C2: orders only, C3: events only
+ */
+static void do_test_share_group_multi_topic_assignment(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *sub_both, *sub_orders, *sub_events;
+ rd_kafka_topic_partition_list_t *c1_assign, *c2_assign, *c3_assign;
+ rd_kafka_t *c1, *c2, *c3;
+ const char *topic_orders = "test-orders";
+ const char *topic_events = "test-events";
+ const char *group = "test-share-group-multi";
+ int total_orders, total_events;
+
+ SUB_TEST_QUICK();
+
+ /* Setup: orders (4 partitions), events (2 partitions) */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic_orders, 4, 1);
+ rd_kafka_mock_topic_create(mcluster, topic_events, 2, 1);
+
+ sub_both = rd_kafka_topic_partition_list_new(2);
+ rd_kafka_topic_partition_list_add(sub_both, topic_orders,
+ RD_KAFKA_PARTITION_UA);
+ rd_kafka_topic_partition_list_add(sub_both, topic_events,
+ RD_KAFKA_PARTITION_UA);
+
+ sub_orders = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(sub_orders, topic_orders,
+ RD_KAFKA_PARTITION_UA);
+
+ sub_events = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(sub_events, topic_events,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ /* C1 joins (both topics) - should get all 6 partitions */
+ c1 = create_share_consumer(bootstraps, group);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, sub_both));
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_ASSERT(c1_assign->cnt == 6,
+ "C1 should have all 6 partitions, got %d", c1_assign->cnt);
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+
+ /* C2 joins (orders only) - orders should split */
+ c2 = create_share_consumer(bootstraps, group);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, sub_orders));
+ wait_share_heartbeats(mcluster, 3, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+ rd_kafka_consumer_poll(c2, 2000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+
+ total_orders = count_topic_partitions(c1_assign, topic_orders) +
+ count_topic_partitions(c2_assign, topic_orders);
+ total_events = count_topic_partitions(c1_assign, topic_events) +
+ count_topic_partitions(c2_assign, topic_events);
+
+ TEST_ASSERT(total_orders == 4, "Total orders should be 4, got %d",
+ total_orders);
+ TEST_ASSERT(total_events == 2, "Total events should be 2, got %d",
+ total_events);
+ TEST_ASSERT(count_topic_partitions(c2_assign, topic_orders) > 0,
+ "C2 should have at least 1 orders partition");
+
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+
+ /* C3 joins (events only) - events should split */
+ c3 = create_share_consumer(bootstraps, group);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c3, sub_events));
+ wait_share_heartbeats(mcluster, 5, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+ rd_kafka_consumer_poll(c2, 2000);
+ rd_kafka_consumer_poll(c3, 2000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c3, &c3_assign));
+
+ total_orders = count_topic_partitions(c1_assign, topic_orders) +
+ count_topic_partitions(c2_assign, topic_orders) +
+ count_topic_partitions(c3_assign, topic_orders);
+ total_events = count_topic_partitions(c1_assign, topic_events) +
+ count_topic_partitions(c2_assign, topic_events) +
+ count_topic_partitions(c3_assign, topic_events);
+
+ TEST_ASSERT(total_orders == 4, "Total orders should be 4, got %d",
+ total_orders);
+ TEST_ASSERT(total_events == 2, "Total events should be 2, got %d",
+ total_events);
+ TEST_ASSERT(count_topic_partitions(c3_assign, topic_events) > 0,
+ "C3 should have at least 1 events partition");
+
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+ rd_kafka_topic_partition_list_destroy(c3_assign);
+
+ /* C1 leaves - C2 should get all orders, C3 all events */
+ rd_kafka_consumer_close(c1);
+ rd_kafka_destroy(c1);
+
+ rd_kafka_consumer_poll(c2, 6000);
+ rd_kafka_consumer_poll(c3, 6000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c3, &c3_assign));
+
+ TEST_ASSERT(count_topic_partitions(c2_assign, topic_orders) == 4,
+ "C2 should have all 4 orders partitions, got %d",
+ count_topic_partitions(c2_assign, topic_orders));
+ TEST_ASSERT(count_topic_partitions(c3_assign, topic_events) == 2,
+ "C3 should have all 2 events partitions, got %d",
+ count_topic_partitions(c3_assign, topic_events));
+
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+ rd_kafka_topic_partition_list_destroy(c3_assign);
+
+ /* C2 leaves - C3 should still have events only */
+ rd_kafka_consumer_close(c2);
+ rd_kafka_destroy(c2);
+
+ rd_kafka_consumer_poll(c3, 6000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c3, &c3_assign));
+ TEST_ASSERT(count_topic_partitions(c3_assign, topic_events) == 2,
+ "C3 should still have 2 events partitions, got %d",
+ count_topic_partitions(c3_assign, topic_events));
+ TEST_ASSERT(count_topic_partitions(c3_assign, topic_orders) == 0,
+ "C3 should have 0 orders partitions (not subscribed), "
+ "got %d",
+ count_topic_partitions(c3_assign, topic_orders));
+ rd_kafka_topic_partition_list_destroy(c3_assign);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c3);
+ rd_kafka_destroy(c3);
+
+ rd_kafka_topic_partition_list_destroy(sub_both);
+ rd_kafka_topic_partition_list_destroy(sub_orders);
+ rd_kafka_topic_partition_list_destroy(sub_events);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Test fatal error injection for ShareGroupHeartbeat.
+ *
+ * When a fatal exception occurs during heartbeat, the consumer should
+ * transition to fatal state and no longer be usable.
+ */
+static void do_test_share_group_error_injection(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t fatal_err;
+ char errstr[256];
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-errors";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject a fatal error (INVALID_REQUEST) during heartbeat.
+ * This matches testFailureOnFatalException which verifies
+ * transitionToFatal() is called on fatal heartbeat errors. */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_INVALID_REQUEST, 0);
+
+ /* Poll - consumer should enter fatal state */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify consumer entered fatal state */
+ fatal_err = rd_kafka_fatal_error(c, errstr, sizeof(errstr));
+ TEST_ASSERT(fatal_err != RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Expected consumer to be in fatal state after "
+ "INVALID_REQUEST error");
+ TEST_SAY("Consumer entered fatal state: %s (%s)\n",
+ rd_kafka_err2str(fatal_err), errstr);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Test network timeout for ShareGroupHeartbeat.
+ *
+ * When a heartbeat times out due to network latency, the consumer should
+ * handle the timeout and retry with backoff, eventually recovering.
+ */
+static void do_test_share_group_rtt_injection(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ rd_kafka_conf_t *conf;
+ char errstr[512];
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-rtt";
+
+ SUB_TEST();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ /* Create consumer with short socket timeout so RTT injection
+ * causes an actual timeout. Default is 60s which is too long. */
+ test_conf_init(&conf, NULL, 0);
+ test_conf_set(conf, "bootstrap.servers", bootstraps);
+ test_conf_set(conf, "group.id", group);
+ test_conf_set(conf, "share.consumer", "true");
+ test_conf_set(conf, "group.protocol", "consumer");
+ test_conf_set(conf, "auto.offset.reset", "earliest");
+ test_conf_set(conf, "socket.timeout.ms", "3000");
+
+ c = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr));
+ TEST_ASSERT(c != NULL, "Failed to create share consumer: %s", errstr);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject RTT larger than socket.timeout.ms to cause a real timeout.
+ * The Java test verifies TimeoutException + backoff retry. */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_NO_ERROR, 5000);
+
+ /* Poll through the timeout period - consumer should recover */
+ rd_kafka_consumer_poll(c, 5000);
+
+ /* Verify heartbeats resumed after timeout recovery */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 1000);
+ TEST_ASSERT(found_heartbeats >= 1,
+ "Expected heartbeats to resume after timeout, got %d",
+ found_heartbeats);
+
+ /* Poll more to allow assignment to be restored */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify consumer recovered and still has assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after timeout recovery, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Test session timeout for ShareGroupHeartbeat.
+ *
+ * Tests that the mock broker correctly times out members that stop
+ * heartbeating. Uses a short session timeout (3000ms) and verifies:
+ * - Member is removed after timeout
+ * - Remaining members get reassigned partitions
+ */
+static void do_test_share_group_session_timeout(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_topic_partition_list_t *c1_assign, *c2_assign;
+ rd_kafka_t *c1, *c2;
+ int c1_initial, c2_initial;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-timeout";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 4, 1);
+
+ /* Set short session timeout on mock broker */
+ rd_kafka_mock_sharegroup_set_session_timeout(mcluster, 3000);
+
+ c1 = create_share_consumer(bootstraps, group);
+ c2 = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, subscription));
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for both to join and rebalance to complete */
+ wait_share_heartbeats(mcluster, 3, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+ rd_kafka_consumer_poll(c2, 2000);
+
+ /* Verify initial distribution */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ c1_initial = c1_assign->cnt;
+ c2_initial = c2_assign->cnt;
+ TEST_ASSERT(c1_initial + c2_initial == 4,
+ "Total should be 4 partitions, got %d",
+ c1_initial + c2_initial);
+ TEST_ASSERT(c1_initial > 0 && c2_initial > 0,
+ "Both consumers should have partitions");
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+
+ /* Destroy C2 without close to simulate crash */
+ rd_kafka_destroy(c2);
+
+ /* Poll C1 for 5 seconds - enough for C2 to timeout */
+ rd_kafka_consumer_poll(c1, 5000);
+
+ /* Verify C1 got all partitions after C2 timed out */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_ASSERT(c1_assign->cnt == 4,
+ "C1 should have all 4 partitions after C2 timeout, got %d",
+ c1_assign->cnt);
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+
+ rd_kafka_consumer_close(c1);
+ rd_kafka_destroy(c1);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Test target assignment API for ShareGroupHeartbeat.
+ *
+ * Tests that the mock broker can apply manual target assignments:
+ * 1. Two consumers join and get automatic assignment (2 partitions each)
+ * 2. Retrieve member IDs using rd_kafka_mock_sharegroup_get_member_ids()
+ * 3. Set manual target assignment: C1 gets all 4 partitions, C2 gets none
+ * 4. Verify consumers receive the manual assignment
+ */
+static void do_test_share_group_target_assignment(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_topic_partition_list_t *c1_assign, *c2_assign;
+ rd_kafka_topic_partition_list_t *target_c1, *target_c2;
+ rd_kafka_topic_partition_list_t *assignments[2];
+ rd_kafka_t *c1, *c2;
+ char **member_ids;
+ size_t member_cnt;
+ rd_kafka_resp_err_t err;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-target";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 4, 1);
+
+ c1 = create_share_consumer(bootstraps, group);
+ c2 = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, subscription));
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for both to join and rebalance to complete */
+ wait_share_heartbeats(mcluster, 3, 500);
+ rd_kafka_consumer_poll(c1, 3000);
+ rd_kafka_consumer_poll(c2, 3000);
+
+ /* Verify initial automatic assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ TEST_ASSERT(c1_assign->cnt + c2_assign->cnt == 4,
+ "Total should be 4 partitions, got %d",
+ c1_assign->cnt + c2_assign->cnt);
+ TEST_ASSERT(c1_assign->cnt > 0 && c2_assign->cnt > 0,
+ "Both consumers should have partitions initially");
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+
+ /* Retrieve member IDs */
+ err = rd_kafka_mock_sharegroup_get_member_ids(mcluster, group,
+ &member_ids, &member_cnt);
+ TEST_ASSERT(err == RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Expected no error, got %s", rd_kafka_err2str(err));
+ TEST_ASSERT(member_cnt == 2, "Expected 2 members, got %zu", member_cnt);
+
+ /* Set manual target assignment: all to first member */
+ target_c1 = rd_kafka_topic_partition_list_new(4);
+ rd_kafka_topic_partition_list_add(target_c1, topic, 0);
+ rd_kafka_topic_partition_list_add(target_c1, topic, 1);
+ rd_kafka_topic_partition_list_add(target_c1, topic, 2);
+ rd_kafka_topic_partition_list_add(target_c1, topic, 3);
+
+ target_c2 = rd_kafka_topic_partition_list_new(0);
+
+ assignments[0] = target_c1;
+ assignments[1] = target_c2;
+
+ rd_kafka_mock_sharegroup_target_assignment(
+ mcluster, group, (const char **)member_ids, assignments, 2);
+
+ rd_kafka_topic_partition_list_destroy(target_c1);
+ rd_kafka_topic_partition_list_destroy(target_c2);
+
+ /* Poll to receive new assignment */
+ rd_kafka_consumer_poll(c1, 6000);
+ rd_kafka_consumer_poll(c2, 6000);
+
+ /* Verify manual assignment was applied */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+
+ TEST_ASSERT(c1_assign->cnt + c2_assign->cnt == 4,
+ "Total should still be 4 partitions, got %d",
+ c1_assign->cnt + c2_assign->cnt);
+ TEST_ASSERT((c1_assign->cnt == 4 && c2_assign->cnt == 0) ||
+ (c1_assign->cnt == 0 && c2_assign->cnt == 4),
+ "Expected one consumer to have all 4 partitions and the "
+ "other to have 0, got C1=%d, C2=%d",
+ c1_assign->cnt, c2_assign->cnt);
+
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+
+ /* Free member IDs */
+ rd_free(member_ids[0]);
+ rd_free(member_ids[1]);
+ rd_free(member_ids);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c1);
+ rd_kafka_consumer_close(c2);
+ rd_kafka_destroy(c1);
+ rd_kafka_destroy(c2);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief UNKNOWN_MEMBER_ID error handling.
+ *
+ * When a consumer receives UNKNOWN_MEMBER_ID error, it should rejoin
+ * with epoch=0 (fresh join).
+ *
+ * NOT YET COMPATIBLE: UNKNOWN_MEMBER_ID triggers an incorrect assert in
+ * development builds (rdkafka_cgrp.c:6631). Pending fix by Pranav.
+ * See sghb_test_discrepancies.txt #3.
+ */
+static void do_test_unknown_member_id_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-unknown-member";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject UNKNOWN_MEMBER_ID error */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID, 0);
+
+ /* Poll - consumer should handle error and rejoin */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify heartbeats continue (rejoin happened) */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 500);
+ TEST_ASSERT(found_heartbeats >= 1,
+ "Expected heartbeats to continue after UNKNOWN_MEMBER_ID, "
+ "got %d",
+ found_heartbeats);
+
+ /* Verify consumer eventually gets assignment back */
+ rd_kafka_consumer_poll(c, 2000);
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after rejoin, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief FENCED_MEMBER_EPOCH error handling.
+ *
+ * When a consumer receives FENCED_MEMBER_EPOCH error, it should be fenced
+ * and then rejoin with epoch=0.
+ */
+static void do_test_fenced_member_epoch_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-fenced";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject FENCED_MEMBER_EPOCH error */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH, 0);
+
+ /* Poll - consumer should handle error and rejoin */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify heartbeats continue (rejoin happened) */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 500);
+ TEST_ASSERT(
+ found_heartbeats >= 1,
+ "Expected heartbeats to continue after FENCED_MEMBER_EPOCH, "
+ "got %d",
+ found_heartbeats);
+
+ /* Verify consumer eventually gets assignment back */
+ rd_kafka_consumer_poll(c, 2000);
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after rejoin, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief COORDINATOR_NOT_AVAILABLE error handling.
+ *
+ * When a consumer receives COORDINATOR_NOT_AVAILABLE, it should retry
+ * (retriable error).
+ */
+static void do_test_coordinator_not_available_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-coord-unavail";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject COORDINATOR_NOT_AVAILABLE error (transient) */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, 0);
+
+ /* Poll - consumer should handle transient error and retry */
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify heartbeats continue after transient error */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 500);
+ TEST_ASSERT(
+ found_heartbeats >= 1,
+ "Expected heartbeats to continue after COORDINATOR_NOT_AVAILABLE, "
+ "got %d",
+ found_heartbeats);
+
+ /* Verify consumer still has assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after retry, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief NOT_COORDINATOR error handling.
+ *
+ * When a consumer receives NOT_COORDINATOR, it should find a new coordinator.
+ */
+static void do_test_not_coordinator_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-not-coord";
+
+ SUB_TEST();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject NOT_COORDINATOR error */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_NOT_COORDINATOR, 0);
+
+ /* Poll - consumer should find new coordinator and continue.
+ * NOT_COORDINATOR triggers coordinator rediscovery which may take
+ * longer than COORDINATOR_NOT_AVAILABLE. */
+ rd_kafka_consumer_poll(c, 5000);
+
+ /* Verify heartbeats continue after finding coordinator */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 1000);
+ TEST_ASSERT(found_heartbeats >= 1,
+ "Expected heartbeats to continue after NOT_COORDINATOR, "
+ "got %d",
+ found_heartbeats);
+
+ /* Verify consumer still has assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after finding coordinator, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief GROUP_AUTHORIZATION_FAILED error handling (fatal).
+ *
+ * When a consumer receives GROUP_AUTHORIZATION_FAILED, it should treat
+ * it as a fatal error.
+ */
+static void do_test_group_authorization_failed_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t fatal_err;
+ char errstr[256];
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-auth-failed";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Inject GROUP_AUTHORIZATION_FAILED error (fatal) */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED, 0);
+
+ /* Poll - should trigger fatal error */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify consumer entered fatal state */
+ fatal_err = rd_kafka_fatal_error(c, errstr, sizeof(errstr));
+ TEST_ASSERT(fatal_err != RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Expected consumer to be in fatal state after "
+ "GROUP_AUTHORIZATION_FAILED");
+ TEST_SAY("Consumer entered fatal state: %s (%s)\n",
+ rd_kafka_err2str(fatal_err), errstr);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief GROUP_MAX_SIZE_REACHED error handling.
+ *
+ * When a new member tries to join and gets GROUP_MAX_SIZE_REACHED,
+ * the error should be treated as fatal for that consumer.
+ */
+static void do_test_group_max_size_reached_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c1, *c2;
+ rd_kafka_resp_err_t fatal_err;
+ char errstr[256];
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-max-size";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 4, 1);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ /* First consumer joins successfully */
+ c1 = create_share_consumer(bootstraps, group);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, subscription));
+
+ /* Wait for c1 to fully join and stabilize */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &assignment));
+ TEST_ASSERT(assignment->cnt == 4,
+ "Expected c1 to have 4 partitions, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Push multiple GROUP_MAX_SIZE_REACHED errors so that even if
+ * c1's regular heartbeat consumes some, c2's join heartbeat
+ * will also get one. The Java test uses server-side maxSize=1
+ * config; we simulate by injecting errors for all heartbeats. */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 5,
+ RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED, 0);
+
+ /* Create second consumer - should be rejected */
+ c2 = create_share_consumer(bootstraps, group);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, subscription));
+
+ /* Poll c2 - should get fatal error */
+ rd_kafka_consumer_poll(c2, 3000);
+
+ /* Verify c2 entered fatal state */
+ fatal_err = rd_kafka_fatal_error(c2, errstr, sizeof(errstr));
+ TEST_ASSERT(fatal_err != RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Expected c2 to be in fatal state after "
+ "GROUP_MAX_SIZE_REACHED");
+ TEST_SAY("c2 correctly rejected with fatal error: %s (%s)\n",
+ rd_kafka_err2str(fatal_err), errstr);
+
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c1);
+ rd_kafka_consumer_close(c2);
+ rd_kafka_destroy(c1);
+ rd_kafka_destroy(c2);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Member rejoin with epoch zero.
+ *
+ * A member in stable state (epoch > 0) that sends heartbeat with epoch=0
+ * should be treated as a rejoin and assigned a new member ID.
+ */
+static void do_test_member_rejoin_with_epoch_zero(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-rejoin";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment (member is now in stable state) */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Force a rejoin by injecting UNKNOWN_MEMBER_ID error
+ * This will cause client to rejoin with epoch=0 */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID, 0);
+
+ /* Poll - consumer should rejoin with epoch=0 */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify rejoin heartbeats */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 500);
+ TEST_ASSERT(found_heartbeats >= 1, "Expected rejoin heartbeats, got %d",
+ found_heartbeats);
+
+ /* Verify consumer gets assignment back */
+ rd_kafka_consumer_poll(c, 2000);
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after rejoin, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Leaving member bumps group epoch.
+ *
+ * When a member sends leave heartbeat (epoch=-1), the group epoch should
+ * be bumped and remaining members should get updated assignment.
+ */
+static void do_test_leaving_member_bumps_group_epoch(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_topic_partition_list_t *c1_assign, *c2_assign;
+ rd_kafka_t *c1, *c2;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-leave-epoch";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 4, 1);
+
+ c1 = create_share_consumer(bootstraps, group);
+ c2 = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, subscription));
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for both to join */
+ wait_share_heartbeats(mcluster, 3, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+ rd_kafka_consumer_poll(c2, 2000);
+
+ /* Verify initial distribution */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ TEST_ASSERT(c1_assign->cnt + c2_assign->cnt == 4,
+ "Total should be 4 partitions, got %d",
+ c1_assign->cnt + c2_assign->cnt);
+ TEST_ASSERT(c1_assign->cnt > 0 && c2_assign->cnt > 0,
+ "Both consumers should have partitions");
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+
+ /* C2 leaves (sends epoch=-1 leave heartbeat) */
+ rd_kafka_consumer_close(c2);
+ rd_kafka_destroy(c2);
+
+ /* Poll C1 to receive updated assignment (group epoch bumped) */
+ rd_kafka_consumer_poll(c1, 6000);
+
+ /* Verify C1 got all partitions after C2 left */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_ASSERT(c1_assign->cnt == 4,
+ "C1 should have all 4 partitions after C2 left, got %d",
+ c1_assign->cnt);
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c1);
+ rd_kafka_destroy(c1);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Partition assignment with changing topics.
+ *
+ * Note: This test is limited in mock broker - we can test initial assignment
+ * with multiple topics in subscription, but cannot dynamically add topics.
+ */
+static void do_test_partition_assignment_with_multiple_topics(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ const char *topic1 = "test-multi-topic-1";
+ const char *topic2 = "test-multi-topic-2";
+ const char *group = "test-share-group-multi-topic-sub";
+ int topic1_count = 0, topic2_count = 0, i;
+
+ SUB_TEST_QUICK();
+
+ /* Setup - create two topics */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic1, 3, 1);
+ rd_kafka_mock_topic_create(mcluster, topic2, 2, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ /* Subscribe to both topics */
+ subscription = rd_kafka_topic_partition_list_new(2);
+ rd_kafka_topic_partition_list_add(subscription, topic1,
+ RD_KAFKA_PARTITION_UA);
+ rd_kafka_topic_partition_list_add(subscription, topic2,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify assignment includes partitions from both topics */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 5, "Expected 5 partitions (3+2), got %d",
+ assignment->cnt);
+
+ /* Count partitions per topic */
+ for (i = 0; i < assignment->cnt; i++) {
+ if (strcmp(assignment->elems[i].topic, topic1) == 0)
+ topic1_count++;
+ else if (strcmp(assignment->elems[i].topic, topic2) == 0)
+ topic2_count++;
+ }
+ TEST_ASSERT(topic1_count == 3,
+ "Expected 3 partitions from topic1, got %d", topic1_count);
+ TEST_ASSERT(topic2_count == 2,
+ "Expected 2 partitions from topic2, got %d", topic2_count);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Multiple members partition distribution.
+ *
+ * N members join group subscribed to topic with M partitions.
+ * Verify partitions are distributed fairly (all members get some).
+ * Note: Share groups may allow the same partition to be assigned to
+ * multiple consumers, so we check for fair distribution rather than
+ * exclusive assignment.
+ */
+static void do_test_multiple_members_partition_distribution(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_topic_partition_list_t *c1_assign, *c2_assign, *c3_assign;
+ rd_kafka_t *c1, *c2, *c3;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-distribution";
+ int total_partitions;
+
+ SUB_TEST_QUICK();
+
+ /* Setup - 6 partitions, 3 consumers */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 6, 1);
+
+ c1 = create_share_consumer(bootstraps, group);
+ c2 = create_share_consumer(bootstraps, group);
+ c3 = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, subscription));
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, subscription));
+ TEST_CALL_ERR__(rd_kafka_subscribe(c3, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for all to join */
+ wait_share_heartbeats(mcluster, 5, 500);
+ rd_kafka_consumer_poll(c1, 3000);
+ rd_kafka_consumer_poll(c2, 3000);
+ rd_kafka_consumer_poll(c3, 3000);
+
+ /* Get assignments */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c3, &c3_assign));
+
+ total_partitions = c1_assign->cnt + c2_assign->cnt + c3_assign->cnt;
+
+ /* In share groups, partitions may be assigned to multiple consumers.
+ * Each consumer should have at least 1 partition, and total should
+ * be at least 6 (covering all partitions). */
+ TEST_ASSERT(c1_assign->cnt >= 1,
+ "Expected c1 to have at least 1 partition, got %d",
+ c1_assign->cnt);
+ TEST_ASSERT(c2_assign->cnt >= 1,
+ "Expected c2 to have at least 1 partition, got %d",
+ c2_assign->cnt);
+ TEST_ASSERT(c3_assign->cnt >= 1,
+ "Expected c3 to have at least 1 partition, got %d",
+ c3_assign->cnt);
+ TEST_ASSERT(total_partitions >= 6,
+ "Expected at least 6 total partition assignments, got %d",
+ total_partitions);
+
+ TEST_SAY("Partition distribution: c1=%d, c2=%d, c3=%d (total=%d)\n",
+ c1_assign->cnt, c2_assign->cnt, c3_assign->cnt,
+ total_partitions);
+
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+ rd_kafka_topic_partition_list_destroy(c3_assign);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c1);
+ rd_kafka_consumer_close(c2);
+ rd_kafka_consumer_close(c3);
+ rd_kafka_destroy(c1);
+ rd_kafka_destroy(c2);
+ rd_kafka_destroy(c3);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Heartbeat successful response completes leave.
+ *
+ * When a member sends leave heartbeat (epoch=-1), verify successful
+ * response completes the leave.
+ */
+static void do_test_leave_heartbeat_completes_successfully(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t err;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-leave-success";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Leave group - should send leave heartbeat and complete.
+ * Note: After close(), we cannot call rd_kafka_assignment() anymore
+ * as the broker handle is destroyed. */
+ err = rd_kafka_consumer_close(c);
+ TEST_ASSERT(!err, "Expected close to succeed, got %s",
+ rd_kafka_err2str(err));
+
+ /* Cleanup */
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Heartbeat failed response during leave still completes.
+ *
+ * When a member sends leave heartbeat and gets an error response,
+ * the leave should still complete (best effort).
+ */
+static void do_test_leave_heartbeat_completes_on_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t err;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-leave-error";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject error for the leave heartbeat */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, 0);
+
+ /* Leave group - should still complete despite error (best effort).
+ * The key behavior: close() must not hang even when the leave
+ * heartbeat gets an error response. */
+ err = rd_kafka_consumer_close(c);
+ /* Close completed (didn't hang) - this is the primary assertion.
+ * The return code may vary depending on whether the error was
+ * processed during leave. */
+ TEST_SAY("Leave completed with: %s (didn't hang - correct)\n",
+ rd_kafka_err2str(err));
+
+ /* Cleanup */
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Subscription change updates assignment.
+ *
+ * Consumer subscribed to topic A, change subscription to topic B,
+ * verify assignment updates.
+ */
+static void do_test_subscription_change(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ int found_topicA = 0, found_topicB = 0, i;
+ const char *topicA = "test-sub-change-topic-A";
+ const char *topicB = "test-sub-change-topic-B";
+ const char *group = "test-share-group-sub-change";
+
+ SUB_TEST();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topicA, 2, 1);
+ rd_kafka_mock_topic_create(mcluster, topicB, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ /* First subscription: topic A */
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topicA,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for assignment to topic A */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify assignment has topic A only */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 2,
+ "Expected 2 partitions from topicA, got %d",
+ assignment->cnt);
+ for (i = 0; i < assignment->cnt; i++) {
+ TEST_ASSERT(strcmp(assignment->elems[i].topic, topicA) == 0,
+ "Expected topicA, got %s",
+ assignment->elems[i].topic);
+ }
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Change subscription to topic B */
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topicB,
+ RD_KAFKA_PARTITION_UA);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for assignment update */
+ rd_kafka_consumer_poll(c, 3000);
+ wait_share_heartbeats(mcluster, 2, 500);
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify assignment now has topic B only */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ found_topicA = 0;
+ found_topicB = 0;
+ for (i = 0; i < assignment->cnt; i++) {
+ if (strcmp(assignment->elems[i].topic, topicA) == 0)
+ found_topicA++;
+ else if (strcmp(assignment->elems[i].topic, topicB) == 0)
+ found_topicB++;
+ }
+ TEST_ASSERT(found_topicA == 0,
+ "Expected 0 partitions from topicA after change, got %d",
+ found_topicA);
+ TEST_ASSERT(found_topicB == 3,
+ "Expected 3 partitions from topicB after change, got %d",
+ found_topicB);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief GROUP_ID_NOT_FOUND while unsubscribed is benign.
+ *
+ * When a member that has already unsubscribed receives GROUP_ID_NOT_FOUND,
+ * it should be treated as benign (the group may have been auto-deleted).
+ * This should NOT cause a fatal error.
+ */
+static void do_test_group_id_not_found_while_unsubscribed(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t err, fatal_err;
+ char errstr[256];
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-id-not-found-unsub";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Unsubscribe first to transition to unsubscribed state.
+ * The Java test has member in UNSUBSCRIBED state when the
+ * error arrives. */
+ TEST_CALL_ERR__(rd_kafka_unsubscribe(c));
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Now inject GROUP_ID_NOT_FOUND.
+ * Since the member is unsubscribed, this should be benign. */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 3,
+ RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND, 0);
+
+ /* Poll to process the error */
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify consumer is NOT in fatal state - error should be benign */
+ fatal_err = rd_kafka_fatal_error(c, errstr, sizeof(errstr));
+ TEST_ASSERT(fatal_err == RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Expected no fatal error when GROUP_ID_NOT_FOUND arrives "
+ "while unsubscribed, but got: %s (%s)",
+ rd_kafka_err2str(fatal_err), errstr);
+
+ /* Close consumer */
+ err = rd_kafka_consumer_close(c);
+ TEST_SAY("Close returned: %s\n", rd_kafka_err2str(err));
+
+ /* Cleanup */
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief GROUP_ID_NOT_FOUND while stable is fatal.
+ *
+ * When an active member (epoch > 0) receives GROUP_ID_NOT_FOUND,
+ * it should be treated as a fatal error (group unexpectedly deleted).
+ *
+ * NOT YET COMPATIBLE: GROUP_ID_NOT_FOUND is not in the SGHB fatal error
+ * list in rdkafka_cgrp.c. It is treated as permanent non-fatal instead.
+ * See sghb_test_discrepancies.txt #2.
+ */
+static void do_test_group_id_not_found_while_stable_is_fatal(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t fatal_err;
+ char errstr[256];
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-id-not-found-stable";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment - member is in stable state */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject GROUP_ID_NOT_FOUND for an active/stable member.
+ * This should be treated as fatal (group unexpectedly deleted). */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND, 0);
+
+ /* Poll - should trigger fatal error */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Check if consumer entered fatal state.
+ * KNOWN ISSUE: GROUP_ID_NOT_FOUND is not in the SGHB fatal error
+ * list in rdkafka_cgrp.c. It falls through to the default case
+ * and is treated as a permanent (non-fatal) error.
+ * See sghb_test_discrepancies.txt for details. */
+ fatal_err = rd_kafka_fatal_error(c, errstr, sizeof(errstr));
+ if (fatal_err != RD_KAFKA_RESP_ERR_NO_ERROR)
+ TEST_SAY("Consumer entered fatal state: %s (%s)\n",
+ rd_kafka_err2str(fatal_err), errstr);
+ else
+ TEST_SAY(
+ "KNOWN ISSUE: GROUP_ID_NOT_FOUND while stable "
+ "did not trigger fatal error "
+ "(see sghb_test_discrepancies.txt)\n");
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief INVALID_REQUEST error handling.
+ *
+ * When a consumer receives INVALID_REQUEST error, it should be treated
+ * as a fatal error.
+ */
+static void do_test_invalid_request_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t fatal_err;
+ char errstr[256];
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-invalid-request";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Inject INVALID_REQUEST error (fatal) */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_INVALID_REQUEST, 0);
+
+ /* Poll - should trigger fatal error */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify consumer entered fatal state */
+ fatal_err = rd_kafka_fatal_error(c, errstr, sizeof(errstr));
+ TEST_ASSERT(fatal_err != RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Expected consumer to be in fatal state after "
+ "INVALID_REQUEST");
+ TEST_SAY("Consumer entered fatal state: %s (%s)\n",
+ rd_kafka_err2str(fatal_err), errstr);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief UNSUPPORTED_VERSION error handling.
+ *
+ * When a consumer receives UNSUPPORTED_VERSION error, it should be
+ * treated as a fatal error.
+ */
+static void do_test_unsupported_version_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t fatal_err;
+ char errstr[256];
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-unsupported-version";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Inject UNSUPPORTED_VERSION error (fatal) */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION, 0);
+
+ /* Poll - should trigger fatal error */
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify consumer entered fatal state */
+ fatal_err = rd_kafka_fatal_error(c, errstr, sizeof(errstr));
+ TEST_ASSERT(fatal_err != RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Expected consumer to be in fatal state after "
+ "UNSUPPORTED_VERSION");
+ TEST_SAY("Consumer entered fatal state: %s (%s)\n",
+ rd_kafka_err2str(fatal_err), errstr);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief COORDINATOR_LOAD_IN_PROGRESS error handling.
+ *
+ * When a consumer receives COORDINATOR_LOAD_IN_PROGRESS, it should
+ * retry with backoff (transient error).
+ */
+static void do_test_coordinator_load_in_progress_error(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-coord-load";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Inject COORDINATOR_LOAD_IN_PROGRESS error (transient) */
+ rd_kafka_mock_broker_push_request_error_rtts(
+ mcluster, 1, RD_KAFKAP_ShareGroupHeartbeat, 1,
+ RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS, 0);
+
+ /* Poll - consumer should handle transient error and retry */
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify heartbeats continue after transient error */
+ found_heartbeats = wait_share_heartbeats(mcluster, 2, 500);
+ TEST_ASSERT(found_heartbeats >= 1,
+ "Expected heartbeats to continue after "
+ "COORDINATOR_LOAD_IN_PROGRESS, got %d",
+ found_heartbeats);
+
+ /* Verify consumer still has assignment */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after retry, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Consumer graceful shutdown during stable state.
+ *
+ * Consumer in stable state leaves group gracefully, sending leave
+ * heartbeat with epoch=-1.
+ */
+static void do_test_graceful_shutdown_stable_state(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ rd_kafka_resp_err_t err;
+ int found_heartbeats;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-graceful-shutdown";
+
+ SUB_TEST_QUICK();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial join and assignment */
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify initial assignment - member is in stable state */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions initially, got %d", assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Record heartbeat count before close */
+ found_heartbeats = wait_share_heartbeats(mcluster, 1, 100);
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ /* Close consumer gracefully - should send leave heartbeat */
+ err = rd_kafka_consumer_close(c);
+ TEST_ASSERT(!err, "Expected close to succeed, got %s",
+ rd_kafka_err2str(err));
+
+ /* Verify leave heartbeat was sent */
+ found_heartbeats = wait_share_heartbeats(mcluster, 1, 500);
+ TEST_SAY("Found %d heartbeats during shutdown\n", found_heartbeats);
+
+ /* Cleanup */
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Consumer resubscribes after unsubscribe.
+ *
+ * Tests the unsubscribe then resubscribe flow.
+ */
+static void do_test_resubscribe_after_unsubscribe(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_t *c;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-resubscribe";
+
+ SUB_TEST();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+
+ c = create_share_consumer(bootstraps, group);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ /* First subscribe */
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ wait_share_heartbeats(mcluster, 1, 500);
+ rd_kafka_consumer_poll(c, 2000);
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions on first subscribe, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Unsubscribe */
+ TEST_SAY("Unsubscribing...\n");
+ TEST_CALL_ERR__(rd_kafka_unsubscribe(c));
+ rd_kafka_consumer_poll(c, 2000);
+
+ /* Verify no assignment after unsubscribe */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_ASSERT(assignment->cnt == 0,
+ "Expected 0 partitions after unsubscribe, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Resubscribe */
+ TEST_SAY("Resubscribing...\n");
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ wait_share_heartbeats(mcluster, 2, 500);
+ rd_kafka_consumer_poll(c, 3000);
+
+ /* Verify assignment restored */
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_SAY("Assignment after resubscribe: %d partitions\n",
+ assignment->cnt);
+ TEST_ASSERT(assignment->cnt == 3,
+ "Expected 3 partitions after resubscribe, got %d",
+ assignment->cnt);
+ rd_kafka_topic_partition_list_destroy(assignment);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Consumer leaves and remaining consumers get reassigned.
+ *
+ * Tests rebalance when a consumer leaves the group.
+ */
+static void do_test_consumer_leave_rebalance(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_topic_partition_list_t *c1_assign, *c2_assign;
+ rd_kafka_t *c1, *c2, *c3;
+ int final_total;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 0);
+ const char *group = "test-share-group-leave-rebalance";
+
+ SUB_TEST();
+
+ /* Setup */
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 6, 1);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ c1 = create_share_consumer(bootstraps, group);
+ c2 = create_share_consumer(bootstraps, group);
+ c3 = create_share_consumer(bootstraps, group);
+
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ /* All three join */
+ TEST_CALL_ERR__(rd_kafka_subscribe(c1, subscription));
+ TEST_CALL_ERR__(rd_kafka_subscribe(c2, subscription));
+ TEST_CALL_ERR__(rd_kafka_subscribe(c3, subscription));
+ rd_kafka_topic_partition_list_destroy(subscription);
+
+ /* Wait for initial balance */
+ wait_share_heartbeats(mcluster, 4, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+ rd_kafka_consumer_poll(c2, 2000);
+ rd_kafka_consumer_poll(c3, 2000);
+
+ wait_share_heartbeats(mcluster, 3, 500);
+ rd_kafka_consumer_poll(c1, 2000);
+ rd_kafka_consumer_poll(c2, 2000);
+ rd_kafka_consumer_poll(c3, 2000);
+
+ /* Get initial assignments */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ TEST_SAY("Initial: c1=%d, c2=%d (before c3 leaves)\n", c1_assign->cnt,
+ c2_assign->cnt);
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+
+ /* c3 leaves */
+ TEST_SAY("Consumer c3 leaving...\n");
+ rd_kafka_consumer_close(c3);
+ rd_kafka_destroy(c3);
+
+ /* Poll remaining consumers for rebalance */
+ wait_share_heartbeats(mcluster, 3, 500);
+ rd_kafka_consumer_poll(c1, 3000);
+ rd_kafka_consumer_poll(c2, 3000);
+
+ /* Get new assignments */
+ TEST_CALL_ERR__(rd_kafka_assignment(c1, &c1_assign));
+ TEST_CALL_ERR__(rd_kafka_assignment(c2, &c2_assign));
+ final_total = c1_assign->cnt + c2_assign->cnt;
+ TEST_SAY("After c3 leave: c1=%d, c2=%d\n", c1_assign->cnt,
+ c2_assign->cnt);
+
+ /* Total should be >= 6 partitions among remaining consumers */
+ TEST_ASSERT(final_total >= 6,
+ "Expected >= 6 partitions after rebalance, got %d",
+ final_total);
+
+ rd_kafka_topic_partition_list_destroy(c1_assign);
+ rd_kafka_topic_partition_list_destroy(c2_assign);
+
+ /* Cleanup */
+ rd_kafka_consumer_close(c1);
+ rd_kafka_consumer_close(c2);
+ rd_kafka_destroy(c1);
+ rd_kafka_destroy(c2);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Test calling close twice on the same consumer
+ */
+static void do_test_double_close(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 1);
+ const char *group_id = topic;
+ rd_kafka_t *c;
+ rd_kafka_topic_partition_list_t *subscription;
+ rd_kafka_resp_err_t err;
+
+ SUB_TEST_QUICK();
+
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ c = create_share_consumer(bootstraps, group_id);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ wait_share_heartbeats(mcluster, 3, 500);
+
+ /* First close - should succeed */
+ err = rd_kafka_consumer_close(c);
+ TEST_ASSERT(!err, "Expected first close to succeed, got %s",
+ rd_kafka_err2str(err));
+
+ /* Second close - should handle gracefully without crashing.
+ * The Java equivalent tests verify the CompletableFuture
+ * completes immediately on double-leave. */
+ err = rd_kafka_consumer_close(c);
+ TEST_SAY("Second close returned: %s (no crash - correct)\n",
+ rd_kafka_err2str(err));
+
+ rd_kafka_topic_partition_list_destroy(subscription);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Test consumer subscribed to a topic with no messages
+ */
+static void do_test_empty_topic_subscription(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ const char *topic = test_mk_topic_name(__FUNCTION__, 1);
+ const char *group_id = topic;
+ rd_kafka_t *c;
+ rd_kafka_topic_partition_list_t *subscription, *assignment;
+ rd_kafka_message_t *msg;
+ int i, msg_count = 0;
+
+ SUB_TEST_QUICK();
+
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+ rd_kafka_mock_topic_create(mcluster, topic, 3, 1);
+ rd_kafka_mock_start_request_tracking(mcluster);
+
+ c = create_share_consumer(bootstraps, group_id);
+
+ subscription = rd_kafka_topic_partition_list_new(1);
+ rd_kafka_topic_partition_list_add(subscription, topic,
+ RD_KAFKA_PARTITION_UA);
+
+ TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription));
+ wait_share_heartbeats(mcluster, 3, 500);
+
+ /* Poll empty topic - should get assignment but no messages */
+ for (i = 0; i < 10; i++) {
+ msg = rd_kafka_consumer_poll(c, 200);
+ if (msg) {
+ if (!msg->err)
+ msg_count++;
+ rd_kafka_message_destroy(msg);
+ }
+ }
+
+ TEST_CALL_ERR__(rd_kafka_assignment(c, &assignment));
+ TEST_SAY("Empty topic: %d partitions, %d messages\n", assignment->cnt,
+ msg_count);
+ TEST_ASSERT(assignment->cnt == 3, "Expected 3 partitions, got %d",
+ assignment->cnt);
+
+ rd_kafka_topic_partition_list_destroy(subscription);
+ rd_kafka_topic_partition_list_destroy(assignment);
+ rd_kafka_consumer_close(c);
+ rd_kafka_destroy(c);
+
+ rd_kafka_mock_stop_request_tracking(mcluster);
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+
+/**
+ * @brief Empty topic list subscription returns INVALID_ARG.
+ *
+ * librdkafka intentionally rejects subscribe() with an empty topic list,
+ * unlike Java which treats it as unsubscribe(). This is an intentional
+ * difference (see sghb_test_discrepancies.txt #1).
+ */
+static void do_test_empty_topic_list_subscription(void) {
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+ rd_kafka_t *c;
+ rd_kafka_topic_partition_list_t *empty_list;
+ rd_kafka_resp_err_t err;
+ const char *group = "test-share-group-empty-topic-list";
+
+ SUB_TEST_QUICK();
+
+ mcluster = test_mock_cluster_new(1, &bootstraps);
+
+ c = create_share_consumer(bootstraps, group);
+
+ /* Subscribe with empty topic list - should return INVALID_ARG */
+ empty_list = rd_kafka_topic_partition_list_new(0);
+ err = rd_kafka_subscribe(c, empty_list);
+ TEST_ASSERT(err == RD_KAFKA_RESP_ERR__INVALID_ARG,
+ "Expected INVALID_ARG from subscribe(empty_list), got %s",
+ rd_kafka_err2str(err));
+ TEST_SAY("subscribe(empty_list) correctly returned %s\n",
+ rd_kafka_err2str(err));
+
+ rd_kafka_topic_partition_list_destroy(empty_list);
+ rd_kafka_destroy(c);
+
+ test_mock_cluster_destroy(mcluster);
+
+ SUB_TEST_PASS();
+}
+
+
+int main_0155_share_group_heartbeat_mock(int argc, char **argv) {
+ TEST_SKIP_MOCK_CLUSTER(0);
+
+ do_test_share_group_heartbeat_basic();
+ do_test_share_group_assignment_rebalance();
+ do_test_share_group_multi_topic_assignment();
+ do_test_share_group_error_injection();
+ do_test_share_group_rtt_injection();
+ do_test_share_group_session_timeout();
+ do_test_share_group_target_assignment();
+
+ do_test_unknown_member_id_error();
+ do_test_fenced_member_epoch_error();
+ do_test_coordinator_not_available_error();
+ do_test_not_coordinator_error();
+ do_test_group_authorization_failed_error();
+ do_test_group_max_size_reached_error();
+ do_test_invalid_request_error();
+ do_test_unsupported_version_error();
+ do_test_coordinator_load_in_progress_error();
+
+ do_test_member_rejoin_with_epoch_zero();
+ do_test_leaving_member_bumps_group_epoch();
+
+ do_test_partition_assignment_with_multiple_topics();
+ do_test_multiple_members_partition_distribution();
+
+ do_test_leave_heartbeat_completes_successfully();
+ do_test_leave_heartbeat_completes_on_error();
+ do_test_graceful_shutdown_stable_state();
+ do_test_consumer_leave_rebalance();
+ do_test_double_close();
+
+ do_test_subscription_change();
+ do_test_resubscribe_after_unsubscribe();
+ do_test_empty_topic_subscription();
+ do_test_empty_topic_list_subscription();
+
+ do_test_group_id_not_found_while_unsubscribed();
+ /* NOT YET COMPATIBLE */
+ /* do_test_group_id_not_found_while_stable_is_fatal(); */
+
+ return 0;
+}
diff --git a/tests/0156-kip932-sharefetch_mockbroker.c b/tests/0156-kip932-sharefetch_mockbroker.c
new file mode 100644
index 0000000000..5ab80b4039
--- /dev/null
+++ b/tests/0156-kip932-sharefetch_mockbroker.c
@@ -0,0 +1,1084 @@
+/*
+ * 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"
+
+#include "../src/rdkafka_proto.h"
+
+/**
+ * @name KIP-932 ShareFetch mock broker tests using the share consumer API.
+ *
+ * Exercises the ShareFetch path via mock broker. There is no coordinator
+ * or ShareAcknowledge support in the mock broker, so group management and
+ * ack-based state transitions are not validated here.
+ */
+
+typedef struct test_ctx_s {
+ rd_kafka_t *producer;
+ rd_kafka_mock_cluster_t *mcluster;
+ const char *bootstraps;
+} test_ctx_t;
+
+static test_ctx_t test_ctx_new(void) {
+ test_ctx_t ctx;
+ rd_kafka_conf_t *conf;
+ char errstr[512];
+
+ memset(&ctx, 0, sizeof(ctx));
+
+ ctx.mcluster = test_mock_cluster_new(3, &ctx.bootstraps);
+
+ TEST_ASSERT(rd_kafka_mock_set_apiversion(
+ ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to enable ShareGroupHeartbeat");
+ TEST_ASSERT(rd_kafka_mock_set_apiversion(ctx.mcluster,
+ RD_KAFKAP_ShareFetch, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to enable ShareFetch");
+
+ /* Create a producer targeting the mock cluster */
+ test_conf_init(&conf, NULL, 0);
+ test_conf_set(conf, "bootstrap.servers", ctx.bootstraps);
+
+ ctx.producer =
+ rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr));
+ TEST_ASSERT(ctx.producer != NULL, "Failed to create producer: %s",
+ errstr);
+
+ return ctx;
+}
+
+static void test_ctx_destroy(test_ctx_t *ctx) {
+ if (ctx->producer)
+ rd_kafka_destroy(ctx->producer);
+ if (ctx->mcluster)
+ test_mock_cluster_destroy(ctx->mcluster);
+ memset(ctx, 0, sizeof(*ctx));
+}
+
+static void
+produce_messages(rd_kafka_t *producer, const char *topic, int msgcnt) {
+ for (int i = 0; i < msgcnt; i++) {
+ char payload[64];
+ snprintf(payload, sizeof(payload), "%s-%d", topic, i);
+ TEST_ASSERT(rd_kafka_producev(
+ producer, RD_KAFKA_V_TOPIC(topic),
+ RD_KAFKA_V_VALUE(payload, strlen(payload)),
+ RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY),
+ RD_KAFKA_V_END) == RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Produce failed");
+ }
+ rd_kafka_flush(producer, 5000);
+}
+
+static rd_kafka_share_t *new_share_consumer(const char *bootstraps,
+ const char *group_id) {
+ rd_kafka_conf_t *conf;
+ rd_kafka_share_t *consumer;
+
+ test_conf_init(&conf, NULL, 0);
+ test_conf_set(conf, "bootstrap.servers", bootstraps);
+ test_conf_set(conf, "group.id", group_id);
+
+ consumer = rd_kafka_share_consumer_new(conf, NULL, 0);
+ TEST_ASSERT(consumer != NULL, "Failed to create share consumer");
+ return consumer;
+}
+
+static void subscribe_topics(rd_kafka_share_t *consumer,
+ const char **topics,
+ int topic_cnt) {
+ rd_kafka_topic_partition_list_t *tpl =
+ rd_kafka_topic_partition_list_new(topic_cnt);
+ for (int i = 0; i < topic_cnt; i++) {
+ rd_kafka_topic_partition_list_add(tpl, topics[i],
+ RD_KAFKA_PARTITION_UA);
+ }
+ TEST_ASSERT(!rd_kafka_share_subscribe(consumer, tpl),
+ "Subscribe failed");
+ rd_kafka_topic_partition_list_destroy(tpl);
+}
+
+static int
+consume_n(rd_kafka_share_t *consumer, int expected, int max_attempts) {
+ int consumed = 0;
+ int attempts = 0;
+
+ while (consumed < expected && attempts < max_attempts) {
+ rd_kafka_message_t *rkmessages[100];
+ size_t rcvd_msgs = 0;
+ rd_kafka_error_t *error;
+
+ error = rd_kafka_share_consume_batch(consumer, 500, rkmessages,
+ &rcvd_msgs);
+ attempts++;
+
+ if (error) {
+ TEST_SAY("consume error: %s\n",
+ rd_kafka_error_string(error));
+ rd_kafka_error_destroy(error);
+ continue;
+ }
+
+ for (size_t i = 0; i < rcvd_msgs; i++) {
+ rd_kafka_message_t *rkmsg = rkmessages[i];
+ if (rkmsg->err) {
+ TEST_SAY("consume error: %s\n",
+ rd_kafka_message_errstr(rkmsg));
+ rd_kafka_message_destroy(rkmsg);
+ continue;
+ }
+ TEST_SAY("Consumed: %.*s\n", (int)rkmsg->len,
+ (const char *)rkmsg->payload);
+ consumed++;
+ rd_kafka_message_destroy(rkmsg);
+ }
+ }
+
+ return consumed;
+}
+
+
+static void do_test_basic_consume(void) {
+ const char *topic = "kip932_pos_basic";
+ const int msgcnt = 5;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, msgcnt);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-pos-basic");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, msgcnt, 50);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == msgcnt, "Expected %d consumed, got %d", msgcnt,
+ consumed);
+ SUB_TEST_PASS();
+}
+
+static void do_test_followup_fetch(void) {
+ const char *topic = "kip932_pos_followup";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, 5);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-pos-followup");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, 3, 30);
+ consumed += consume_n(consumer, 2, 30);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 5, "Expected 5 consumed, got %d", consumed);
+ SUB_TEST_PASS();
+}
+
+static void do_test_multi_partition(void) {
+ const char *topic = "kip932_pos_multi_part";
+ const int msgcnt = 6;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 2, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, msgcnt);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-pos-multipart");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, msgcnt, 60);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == msgcnt, "Expected %d consumed, got %d", msgcnt,
+ consumed);
+ SUB_TEST_PASS();
+}
+
+static void do_test_multi_topic(void) {
+ const char *topic_a = "kip932_pos_topic_a";
+ const char *topic_b = "kip932_pos_topic_b";
+ const char *topics[] = {topic_a, topic_b};
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic A");
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic B");
+
+ produce_messages(ctx.producer, topic_a, 2);
+ produce_messages(ctx.producer, topic_b, 2);
+ consumer = new_share_consumer(ctx.bootstraps, "sg-pos-multitopic");
+ subscribe_topics(consumer, topics, 2);
+ consumed = consume_n(consumer, 4, 40);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 4, "Expected 4 consumed, got %d", consumed);
+ SUB_TEST_PASS();
+}
+
+static void do_test_empty_topic_no_records(void) {
+ const char *topic = "kip932_pos_empty";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-pos-empty");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, 1, 5);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed, got %d", consumed);
+ SUB_TEST_PASS();
+}
+
+static void do_test_negative_sharefetch_error(rd_kafka_resp_err_t err) {
+ const char *topic = "kip932_neg_sharefetch_error";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, 1);
+
+ rd_kafka_mock_push_request_errors(ctx.mcluster, RD_KAFKAP_ShareFetch, 1,
+ err);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-neg-sharefetch");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, 1, 5);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed, got %d", consumed);
+}
+
+static void do_test_sharefetch_invalid_session_epoch(void) {
+ SUB_TEST_QUICK();
+ do_test_negative_sharefetch_error(
+ RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH);
+ SUB_TEST_PASS();
+}
+
+static void do_test_sharefetch_unknown_topic_or_part(void) {
+ SUB_TEST_QUICK();
+ do_test_negative_sharefetch_error(
+ RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART);
+ SUB_TEST_PASS();
+}
+
+static void do_test_sghb_error(rd_kafka_resp_err_t err, int count) {
+ const char *topic = "kip932_neg_sghb";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+ rd_kafka_resp_err_t *errs;
+ int i;
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, 1);
+
+ /* Build an array of 'count' identical errors and push them all.
+ * Using the array variant avoids UB from mismatched varargs count. */
+ errs = malloc(sizeof(*errs) * count);
+ TEST_ASSERT(errs != NULL, "malloc failed");
+ for (i = 0; i < count; i++)
+ errs[i] = err;
+ rd_kafka_mock_push_request_errors_array(
+ ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, (size_t)count, errs);
+ free(errs);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-neg-sghb");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, 1, 5);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed, got %d", consumed);
+}
+
+static void do_test_sghb_coord_unavailable(void) {
+ SUB_TEST_QUICK();
+ do_test_sghb_error(RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, 50);
+ SUB_TEST_PASS();
+}
+
+static void do_test_topic_error(rd_kafka_resp_err_t err) {
+ const char *topic = "kip932_neg_topic_error";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, 1);
+ rd_kafka_mock_topic_set_error(ctx.mcluster, topic, err);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-neg-topicerr");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, 1, 5);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed, got %d", consumed);
+}
+
+static void do_test_topic_error_unknown_topic_or_part(void) {
+ SUB_TEST_QUICK();
+ do_test_topic_error(RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART);
+ SUB_TEST_PASS();
+}
+
+static void do_test_unknown_topic_subscription(void) {
+ const char *topic = "kip932_neg_unknown_topic";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-neg-unknown-topic");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, 1, 5);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed, got %d", consumed);
+ SUB_TEST_PASS();
+}
+
+static void do_test_empty_fetch_no_records(void) {
+ const char *topic = "kip932_neg_empty_fetch";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-neg-empty");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, 1, 5);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed, got %d", consumed);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Verify that ShareFetch rejects requests from an unregistered member
+ * (UNKNOWN_MEMBER_ID), and that after the member re-joins it can
+ * consume again.
+ *
+ * Phase 1: Consumer joins normally via SGHB -> consumes messages OK.
+ * Phase 2: Push SGHB errors -> heartbeats fail -> member expires -> broker
+ * rejects ShareFetch with UNKNOWN_MEMBER_ID.
+ * Phase 3: SGHB errors drain -> member re-joins -> consumes again.
+ */
+static void do_test_member_validation(void) {
+ const char *topic = "kip932_member_validation";
+ const int msgcnt = 4;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed_p1, consumed_p3;
+
+ SUB_TEST();
+
+ /* Short session timeout so the member is evicted quickly once
+ * heartbeats stop succeeding. */
+ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500);
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, msgcnt);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-member-val");
+ subscribe_topics(consumer, &topic, 1);
+
+ /* Phase 1: Consume normally -- member is registered via SGHB. */
+ consumed_p1 = consume_n(consumer, 2, 30);
+ TEST_SAY("member_validation: phase1 consumed %d/2\n", consumed_p1);
+
+ /* Phase 2: Block SGHB so heartbeats fail.
+ * Push enough errors to cover the window while we wait for the
+ * member to be evicted. */
+ rd_kafka_mock_push_request_errors(
+ ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, 20,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE,
+ RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE);
+
+ /* Wait for the member to be evicted (500ms session timeout + margin).
+ */
+ usleep(1500 * 1000);
+
+ /* Phase 3: SGHB errors will eventually drain. Once a SGHB
+ * succeeds, the member re-joins and the remaining records
+ * become fetchable again. */
+ consumed_p3 = consume_n(consumer, 2, 50);
+ TEST_SAY("member_validation: phase3 consumed %d/2\n", consumed_p3);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed_p1 >= 2 && (consumed_p1 + consumed_p3) >= msgcnt,
+ "Expected at least 2+2, got %d+%d", consumed_p1,
+ consumed_p3);
+ SUB_TEST_PASS();
+}
+
+static void do_test_sharefetch_session_expiry_rtt(void) {
+ const char *topic = "kip932_rtt_expiry";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST();
+
+ /* Session timeout must be long enough for normal requests
+ * to complete, but short enough to expire during high RTT. */
+ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 1000);
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, 2);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-rtt-expiry");
+ subscribe_topics(consumer, &topic, 1);
+
+ /* Phase 1: consume one message with normal RTT (no injection). */
+ consumed = consume_n(consumer, 1, 20);
+ TEST_SAY("rtt_expiry: phase1 consumed %d/1\n", consumed);
+
+ /* Phase 2: inject RTT >> session timeout to force session expiry.
+ * All requests to broker 1 now take 3s, but the session
+ * expires after 1s of inactivity. */
+ rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 3000);
+ usleep(2000 * 1000); /* wait for session to expire */
+
+ /* Phase 3: clear RTT and let the consumer recover. */
+ rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 0);
+ consumed += consume_n(consumer, 1, 30);
+ TEST_SAY("rtt_expiry: phase3 consumed %d/2 total\n", consumed);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 2, "Expected 2 consumed, got %d", consumed);
+ SUB_TEST_PASS();
+}
+
+static void do_test_forgotten_topics(void) {
+ const char *topic_a = "kip932_forgotten_a";
+ const char *topic_b = "kip932_forgotten_b";
+ const char *both[] = {topic_a, topic_b};
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic A");
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic B");
+
+ /* Produce 2 messages to each topic */
+ produce_messages(ctx.producer, topic_a, 2);
+ produce_messages(ctx.producer, topic_b, 2);
+
+ /* Subscribe to both topics and consume all 4 messages */
+ consumer = new_share_consumer(ctx.bootstraps, "sg-forgotten");
+ subscribe_topics(consumer, both, 2);
+ consumed = consume_n(consumer, 4, 40);
+ TEST_SAY("forgotten_topics: consumed %d/4 from both topics\n",
+ consumed);
+
+ /* Re-subscribe to only topic_a (topic_b becomes forgotten) */
+ subscribe_topics(consumer, &topic_a, 1);
+
+ /* Produce 2 more messages to topic_a */
+ produce_messages(ctx.producer, topic_a, 2);
+
+ /* Consume the 2 new messages -- only topic_a should deliver */
+ consumed += consume_n(consumer, 2, 30);
+ TEST_SAY("forgotten_topics: consumed %d/6 total after forget\n",
+ consumed);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ /* We expect at least the 4 initial + 2 from topic_a = 6.
+ * Depending on timing the consumer may or may not have already
+ * received all messages from the first round, so we accept >= 4. */
+ TEST_ASSERT(consumed >= 4, "Expected at least 4 consumed, got %d",
+ consumed);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Produce messages one-at-a-time (each flush creates a separate
+ * msgset on the mock partition), then consume and verify all are
+ * received. This validates that the ShareFetch response includes
+ * records from *all* acquired msgsets, not just the first one.
+ */
+static void do_test_multi_batch_consume(void) {
+ const char *topic = "kip932_multi_batch";
+ const int msgcnt = 5;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST_QUICK();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+
+ /* Produce each message individually with a flush in between,
+ * guaranteeing separate msgsets on the mock partition. */
+ for (int i = 0; i < msgcnt; i++) {
+ char payload[64];
+ snprintf(payload, sizeof(payload), "batch-%d", i);
+ TEST_ASSERT(rd_kafka_producev(
+ ctx.producer, RD_KAFKA_V_TOPIC(topic),
+ RD_KAFKA_V_VALUE(payload, strlen(payload)),
+ RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY),
+ RD_KAFKA_V_END) == RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Produce failed");
+ rd_kafka_flush(ctx.producer, 5000);
+ }
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-multi-batch");
+ subscribe_topics(consumer, &topic, 1);
+ consumed = consume_n(consumer, msgcnt, 50);
+ TEST_SAY("multi_batch: consumed %d/%d\n", consumed, msgcnt);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == msgcnt, "Expected %d consumed, got %d", msgcnt,
+ consumed);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Verify that max_delivery_attempts causes records to be archived
+ * after the limit is exceeded. Consumer A acquires all records, then
+ * its session times out (releasing locks). Consumer B acquires them
+ * again, and its session also times out. After the delivery limit is
+ * exhausted, Consumer C should see 0 available records.
+ */
+static void do_test_max_delivery_attempts(void) {
+ const char *topic = "kip932_max_delivery";
+ const int msgcnt = 3;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed_a, consumed_b, consumed_c;
+
+ SUB_TEST();
+
+ /* Set max delivery attempts to 2 and a short session timeout
+ * so locks expire quickly after consumer destruction. */
+ rd_kafka_mock_sharegroup_set_max_delivery_attempts(ctx.mcluster, 2);
+ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500);
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, msgcnt);
+
+ /* Delivery 1: Consumer A acquires and "crashes" (no ack). */
+ consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery");
+ subscribe_topics(consumer, &topic, 1);
+ consumed_a = consume_n(consumer, msgcnt, 50);
+ TEST_SAY("max_delivery: A consumed %d/%d (delivery 1)\n", consumed_a,
+ msgcnt);
+ rd_kafka_share_destroy(consumer);
+ usleep(1500 * 1000); /* wait for lock expiry */
+
+ /* Delivery 2: Consumer B acquires same records again (delivery_count
+ * reaches 2 = limit) and "crashes". */
+ consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery");
+ subscribe_topics(consumer, &topic, 1);
+ consumed_b = consume_n(consumer, msgcnt, 50);
+ TEST_SAY("max_delivery: B consumed %d/%d (delivery 2)\n", consumed_b,
+ msgcnt);
+ rd_kafka_share_destroy(consumer);
+ usleep(1500 * 1000); /* wait for lock expiry */
+
+ /* Delivery 3 attempt: Consumer C should get 0 records because
+ * all records have been archived (delivery_count >= max). */
+ consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery");
+ subscribe_topics(consumer, &topic, 1);
+ consumed_c = consume_n(consumer, 1, 10);
+ TEST_SAY("max_delivery: C consumed %d/0 (should be archived)\n",
+ consumed_c);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed_a == msgcnt && consumed_b == msgcnt &&
+ consumed_c == 0,
+ "Expected A=%d B=%d C=0, got A=%d B=%d C=%d", msgcnt,
+ msgcnt, consumed_a, consumed_b, consumed_c);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Verify that record_lock_duration_ms controls how long acquired
+ * records stay locked, independently of session_timeout_ms.
+ * Sets a short lock duration (300ms) with a longer session timeout
+ * (10s). Consumer A acquires records and "crashes". After the short
+ * lock duration expires, Consumer B should be able to acquire them
+ * even though A's session hasn't timed out yet.
+ */
+static void do_test_record_lock_duration(void) {
+ const char *topic = "kip932_lock_duration";
+ const int msgcnt = 3;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed_a, consumed_b;
+
+ SUB_TEST();
+
+ /* Long session timeout, short record lock duration. */
+ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 10000);
+ rd_kafka_mock_sharegroup_set_record_lock_duration(ctx.mcluster, 300);
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, msgcnt);
+
+ /* Consumer A acquires records, then crashes (no close). */
+ consumer = new_share_consumer(ctx.bootstraps, "sg-lock-duration");
+ subscribe_topics(consumer, &topic, 1);
+ consumed_a = consume_n(consumer, msgcnt, 50);
+ TEST_SAY("lock_duration: A consumed %d/%d\n", consumed_a, msgcnt);
+ rd_kafka_share_destroy(consumer);
+
+ /* Wait for record lock to expire (300ms + margin),
+ * but NOT session timeout (10s). */
+ usleep(800 * 1000);
+
+ /* Consumer B should get the records because locks have expired
+ * even though A's session is still technically alive. */
+ consumer = new_share_consumer(ctx.bootstraps, "sg-lock-duration");
+ subscribe_topics(consumer, &topic, 1);
+ consumed_b = consume_n(consumer, msgcnt, 50);
+ TEST_SAY("lock_duration: B consumed %d/%d\n", consumed_b, msgcnt);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed_a == msgcnt && consumed_b == msgcnt,
+ "Expected A=%d B=%d, got A=%d B=%d", msgcnt, msgcnt,
+ consumed_a, consumed_b);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Multi-consumer lock expiry test.
+ *
+ * Consumer A acquires records, then crashes (destroyed without close).
+ * After the lock expiry timeout, consumer B should be able to pick up
+ * the same records because the proactive lock-expiry scan releases them.
+ */
+static void do_test_multi_consumer_lock_expiry(void) {
+ const char *topic = "kip932_multi_consumer_lock";
+ const int msgcnt = 3;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer_a, *consumer_b;
+ int consumed_a, consumed_b;
+
+ SUB_TEST();
+
+ /* Use a short session/lock timeout so the test runs quickly. */
+ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500);
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, msgcnt);
+
+ /* Consumer A: subscribe and consume all records (acquires locks). */
+ consumer_a =
+ new_share_consumer(ctx.bootstraps, "sg-multi-consumer-lock");
+ subscribe_topics(consumer_a, &topic, 1);
+ consumed_a = consume_n(consumer_a, msgcnt, 50);
+ TEST_SAY("multi_consumer: A consumed %d/%d\n", consumed_a, msgcnt);
+
+ /* Simulate crash: destroy consumer A without calling close.
+ * The session will time out and the proactive lock-expiry
+ * timer will release A's locks. */
+ rd_kafka_share_destroy(consumer_a);
+
+ /* Wait for locks to expire (session_timeout=500ms, add margin). */
+ usleep(1500 * 1000);
+
+ /* Consumer B: joins the same share group, should get the same
+ * records once the locks have been released. */
+ consumer_b =
+ new_share_consumer(ctx.bootstraps, "sg-multi-consumer-lock");
+ subscribe_topics(consumer_b, &topic, 1);
+ consumed_b = consume_n(consumer_b, msgcnt, 50);
+ TEST_SAY("multi_consumer: B consumed %d/%d\n", consumed_b, msgcnt);
+
+ rd_kafka_share_consumer_close(consumer_b);
+ rd_kafka_share_destroy(consumer_b);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed_a == msgcnt && consumed_b == msgcnt,
+ "Expected A=%d B=%d, got A=%d B=%d", msgcnt, msgcnt,
+ consumed_a, consumed_b);
+ SUB_TEST_PASS();
+}
+
+
+/**
+ * @brief Helper for ShareFetch error response tests.
+ *
+ * Injects 1 error of type \p err into the ShareFetch path,
+ * then verifies the consumer returns no records on that fetch.
+ */
+static void do_test_sharefetch_fetch_error(rd_kafka_resp_err_t err) {
+ const char *topic = "kip932_fetch_error";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ rd_kafka_message_t *rkmessages[10];
+ size_t rcvd = 0;
+ int consumed = 0;
+ size_t i;
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, 3);
+
+ /* Push a single error */
+ rd_kafka_mock_push_request_errors(ctx.mcluster, RD_KAFKAP_ShareFetch, 1,
+ err);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-fetch-error");
+ subscribe_topics(consumer, &topic, 1);
+
+ /* Single poll — should return no records */
+ rd_kafka_share_consume_batch(consumer, 2000, rkmessages, &rcvd);
+ for (i = 0; i < rcvd; i++) {
+ if (!rkmessages[i]->err)
+ consumed++;
+ rd_kafka_message_destroy(rkmessages[i]);
+ }
+
+ TEST_SAY("fetch_error(%s): consumed=%d rcvd=%zu\n",
+ rd_kafka_err2name(err), consumed, rcvd);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed with %s, got %d",
+ rd_kafka_err2name(err), consumed);
+}
+
+/**
+ * @brief NOT_LEADER_OR_FOLLOWER in ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_fetch_error_not_leader(void) {
+ SUB_TEST();
+ do_test_sharefetch_fetch_error(
+ RD_KAFKA_RESP_ERR_NOT_LEADER_OR_FOLLOWER);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief UNKNOWN_TOPIC_OR_PARTITION in ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_fetch_error_unknown_topic_or_part(void) {
+ SUB_TEST();
+ do_test_sharefetch_fetch_error(RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief UNKNOWN_TOPIC_ID in ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_fetch_error_unknown_topic_id(void) {
+ SUB_TEST();
+ do_test_sharefetch_fetch_error(RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief FENCED_LEADER_EPOCH in ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_fetch_error_fenced_leader_epoch(void) {
+ SUB_TEST();
+ do_test_sharefetch_fetch_error(RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief UNKNOWN_LEADER_EPOCH in ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_fetch_error_unknown_leader_epoch(void) {
+ SUB_TEST();
+ do_test_sharefetch_fetch_error(RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief TOPIC_AUTHORIZATION_FAILED in ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_topic_authorization_failed(void) {
+ SUB_TEST();
+ do_test_sharefetch_fetch_error(
+ RD_KAFKA_RESP_ERR_TOPIC_AUTHORIZATION_FAILED);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief CORRUPT_MESSAGE (INVALID_MSG) in ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_corrupt_message(void) {
+ SUB_TEST();
+ do_test_sharefetch_fetch_error(RD_KAFKA_RESP_ERR_INVALID_MSG);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Disconnect during ShareFetch -> no records returned.
+ */
+static void do_test_sharefetch_fetch_disconnected(void) {
+ const char *topic = "kip932_disconnect";
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ rd_kafka_message_t *rkmessages[10];
+ size_t rcvd = 0;
+ int consumed = 0;
+ size_t i;
+
+ SUB_TEST();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, 3);
+
+ /* Push a single disconnect */
+ rd_kafka_mock_push_request_errors(ctx.mcluster, RD_KAFKAP_ShareFetch, 1,
+ RD_KAFKA_RESP_ERR__TRANSPORT);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-disconnect");
+ subscribe_topics(consumer, &topic, 1);
+
+ /* Single poll — should return no records on disconnect */
+ rd_kafka_share_consume_batch(consumer, 2000, rkmessages, &rcvd);
+ for (i = 0; i < rcvd; i++) {
+ if (!rkmessages[i]->err)
+ consumed++;
+ rd_kafka_message_destroy(rkmessages[i]);
+ }
+
+ TEST_SAY("fetch_disconnected: consumed=%d rcvd=%zu\n", consumed, rcvd);
+
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == 0, "Expected 0 consumed on disconnect, got %d",
+ consumed);
+ SUB_TEST_PASS();
+}
+
+/**
+ * @brief Fetch records and close implicitly (no explicit ack).
+ *
+ * Verifies that a consumer can fetch records and then close successfully.
+ * The close sends ShareAcknowledge with epoch=-1 to release the session.
+ */
+static void do_test_sharefetch_fetch_and_close_implicit(void) {
+ const char *topic = "kip932_fetch_close";
+ const int msgcnt = 2;
+ test_ctx_t ctx = test_ctx_new();
+ rd_kafka_share_t *consumer;
+ int consumed;
+
+ SUB_TEST();
+
+ TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) ==
+ RD_KAFKA_RESP_ERR_NO_ERROR,
+ "Failed to create mock topic");
+ produce_messages(ctx.producer, topic, msgcnt);
+
+ consumer = new_share_consumer(ctx.bootstraps, "sg-fetch-close");
+ subscribe_topics(consumer, &topic, 1);
+
+ /* Fetch records */
+ consumed = consume_n(consumer, msgcnt, 50);
+ TEST_SAY("fetch_and_close: consumed %d/%d\n", consumed, msgcnt);
+
+ /* Close — sends ShareAcknowledge/ShareFetch with epoch=-1 */
+ rd_kafka_share_consumer_close(consumer);
+ rd_kafka_share_destroy(consumer);
+ test_ctx_destroy(&ctx);
+
+ TEST_ASSERT(consumed == msgcnt, "Expected %d consumed, got %d", msgcnt,
+ consumed);
+ SUB_TEST_PASS();
+}
+
+int main_0156_kip932_sharefetch_mockbroker(int argc, char **argv) {
+ TEST_SKIP_MOCK_CLUSTER(0);
+
+ /* Positive scenarios */
+ do_test_basic_consume();
+ do_test_followup_fetch();
+ do_test_multi_partition();
+ do_test_multi_topic();
+ do_test_empty_topic_no_records();
+ do_test_sharefetch_session_expiry_rtt();
+ do_test_forgotten_topics();
+ do_test_multi_batch_consume();
+ do_test_max_delivery_attempts();
+ do_test_record_lock_duration();
+ do_test_multi_consumer_lock_expiry();
+
+ /* Negative scenarios */
+ do_test_sharefetch_invalid_session_epoch();
+ do_test_sharefetch_unknown_topic_or_part();
+ do_test_sghb_coord_unavailable();
+ do_test_topic_error_unknown_topic_or_part();
+ do_test_unknown_topic_subscription();
+ do_test_empty_fetch_no_records();
+ do_test_member_validation();
+
+ do_test_sharefetch_fetch_error_not_leader();
+ do_test_sharefetch_fetch_error_unknown_topic_or_part();
+ do_test_sharefetch_fetch_error_unknown_topic_id();
+ do_test_sharefetch_fetch_error_fenced_leader_epoch();
+ do_test_sharefetch_fetch_error_unknown_leader_epoch();
+
+ do_test_sharefetch_topic_authorization_failed();
+ do_test_sharefetch_corrupt_message();
+
+ do_test_sharefetch_fetch_disconnected();
+ do_test_sharefetch_fetch_and_close_implicit();
+
+ return 0;
+}
diff --git a/tests/0170-share_consumer_subscription.c b/tests/0170-share_consumer_subscription.c
new file mode 100644
index 0000000000..7673e682c5
--- /dev/null
+++ b/tests/0170-share_consumer_subscription.c
@@ -0,0 +1,892 @@
+/*
+ * 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"
+#include "testshared.h"
+#include "rdkafka.h"
+
+/**
+ * @brief Share consumer subscription tests using operation-based framework.
+ *
+ * This test file uses a declarative, operation-based framework where tests
+ * are defined as sequences of operations. The framework handles:
+ * - Automatic topic name generation
+ * - Topic creation and deletion
+ * - Consumer creation and destruction
+ * - Message production and consumption
+ * - Subscription verification
+ */
+
+
+#define MAX_TOPICS 20
+#define MAX_CONSUMERS 4
+#define MAX_OPS 50
+
+/**
+ * @brief Operation types for subscription tests
+ */
+typedef enum {
+ TEST_OP_END = 0, /**< End of operations marker */
+ TEST_OP_SUBSCRIBE, /**< Subscribe to N new topics */
+ TEST_OP_SUBSCRIBE_ADD, /**< Add N topics to existing subscription */
+ TEST_OP_UNSUBSCRIBE, /**< Unsubscribe from all topics */
+ TEST_OP_RESUBSCRIBE, /**< Replace subscription with N new topics */
+ TEST_OP_PRODUCE, /**< Produce to specified topic set */
+ TEST_OP_CONSUME, /**< Consume messages */
+ TEST_OP_VERIFY_SUB_CNT, /**< Verify subscription count */
+ TEST_OP_DELETE_TOPIC, /**< Delete topic by index */
+ TEST_OP_WAIT, /**< Wait for specified milliseconds */
+ TEST_OP_CREATE_CONSUMER, /**< Create additional consumer */
+ TEST_OP_POLL_NO_SUB, /**< Poll without subscription (edge case) */
+ TEST_OP_CREATE_TOPIC, /**< Create subscribed topics that weren't created
+ */
+ TEST_OP_SUBSCRIBE_EXISTING, /**< Subscribe to already created topics */
+ TEST_OP_PRODUCE_TO_TOPIC, /**< Produce to specific topic index */
+} test_op_type_t;
+
+/**
+ * @brief Flags for operations
+ */
+typedef enum {
+ TEST_OP_F_NONE = 0,
+ TEST_OP_F_SKIP_TOPIC_CREATE = 1 << 0, /**< Don't create topics */
+ TEST_OP_F_PRODUCE_TO_OLD = 1 << 1, /**< Produce to old subscription */
+ TEST_OP_F_VERIFY_NO_OLD_MSGS = 1 << 2, /**< Verify no old messages */
+} test_op_flags_t;
+
+/**
+ * @brief Single operation in a test scenario
+ */
+typedef struct {
+ test_op_type_t op; /**< Operation type */
+ int topic_cnt; /**< Number of topics (SUBSCRIBE/RESUBSCRIBE) */
+ int msgs_per_topic; /**< Messages per topic (PRODUCE) */
+ int expected_msgs; /**< Expected message count (CONSUME, -1=any) */
+ int expected_sub_cnt; /**< Expected subscription count (VERIFY_SUB_CNT)
+ */
+ int topic_idx; /**< Topic index (DELETE_TOPIC) */
+ int wait_ms; /**< Wait time (WAIT) */
+ int consumer_idx; /**< Consumer index (multi-consumer) */
+ int repeat_cnt; /**< Repeat count (SUBSCRIBE/UNSUBSCRIBE) */
+ test_op_flags_t flags; /**< Operation flags */
+} test_op_t;
+
+/**
+ * @brief Test scenario configuration
+ */
+typedef struct {
+ const char *name; /**< Test name for logging */
+ int consumer_cnt; /**< Number of consumers (default: 1) */
+ test_op_t ops[MAX_OPS]; /**< Operations, terminated by TEST_OP_END */
+} test_scenario_t;
+
+/**
+ * @brief Runtime state for test execution
+ */
+typedef struct {
+ /* Consumers */
+ rd_kafka_share_t *consumers[MAX_CONSUMERS];
+ int consumer_cnt;
+
+ /* Topics: all created topics */
+ char *all_topics[MAX_TOPICS];
+ int all_topic_cnt;
+ int msgs_produced[MAX_TOPICS]; /**< Messages produced per topic */
+ rd_bool_t topic_deleted[MAX_TOPICS]; /**< Track deleted topics */
+
+ /* Current subscription tracking per consumer */
+ int sub_start_idx[MAX_CONSUMERS]; /**< Start index in all_topics */
+ int sub_count[MAX_CONSUMERS]; /**< Count of subscribed topics */
+
+ /* Previous subscription (for RESUBSCRIBE verification) */
+ int old_sub_start_idx;
+ int old_sub_count;
+
+ /* Group name */
+ char group_name[128];
+} sub_test_state_t;
+
+
+#define SUBSCRIBE(n) \
+ {.op = TEST_OP_SUBSCRIBE, .topic_cnt = (n), .repeat_cnt = 1}
+#define SUBSCRIBE_REPEAT(n, r) \
+ {.op = TEST_OP_SUBSCRIBE, .topic_cnt = (n), .repeat_cnt = (r)}
+#define SUBSCRIBE_ADD(n) {.op = TEST_OP_SUBSCRIBE_ADD, .topic_cnt = (n)}
+#define SUBSCRIBE_NO_CREATE(n) \
+ {.op = TEST_OP_SUBSCRIBE, \
+ .topic_cnt = (n), \
+ .repeat_cnt = 1, \
+ .flags = TEST_OP_F_SKIP_TOPIC_CREATE}
+#define UNSUBSCRIBE() {.op = TEST_OP_UNSUBSCRIBE, .repeat_cnt = 1}
+#define UNSUBSCRIBE_REPEAT(r) {.op = TEST_OP_UNSUBSCRIBE, .repeat_cnt = (r)}
+#define RESUBSCRIBE(n) {.op = TEST_OP_RESUBSCRIBE, .topic_cnt = (n)}
+#define PRODUCE(msgs) {.op = TEST_OP_PRODUCE, .msgs_per_topic = (msgs)}
+#define PRODUCE_TO_OLD(msgs) \
+ {.op = TEST_OP_PRODUCE, \
+ .msgs_per_topic = (msgs), \
+ .flags = TEST_OP_F_PRODUCE_TO_OLD}
+#define PRODUCE_TO_TOPIC(idx, msgs) \
+ {.op = TEST_OP_PRODUCE_TO_TOPIC, \
+ .topic_idx = (idx), \
+ .msgs_per_topic = (msgs)}
+#define CONSUME(expected) {.op = TEST_OP_CONSUME, .expected_msgs = (expected)}
+#define CONSUME_VERIFY_NO_OLD(expected) \
+ {.op = TEST_OP_CONSUME, \
+ .expected_msgs = (expected), \
+ .flags = TEST_OP_F_VERIFY_NO_OLD_MSGS}
+#define CONSUME_ANY() {.op = TEST_OP_CONSUME, .expected_msgs = -1}
+#define VERIFY_SUB(cnt) \
+ {.op = TEST_OP_VERIFY_SUB_CNT, .expected_sub_cnt = (cnt)}
+#define DELETE_TOPIC(idx) {.op = TEST_OP_DELETE_TOPIC, .topic_idx = (idx)}
+#define WAIT_MS(ms) {.op = TEST_OP_WAIT, .wait_ms = (ms)}
+#define CREATE_CONSUMER(idx) \
+ {.op = TEST_OP_CREATE_CONSUMER, .consumer_idx = (idx)}
+#define CREATE_TOPIC(n) {.op = TEST_OP_CREATE_TOPIC, .topic_cnt = (n)}
+#define SUBSCRIBE_EXISTING() {.op = TEST_OP_SUBSCRIBE_EXISTING, .repeat_cnt = 1}
+#define POLL_NO_SUB() {.op = TEST_OP_POLL_NO_SUB}
+#define TEST_OPS_END() {.op = TEST_OP_END}
+
+
+/**
+ * @brief Set group config to earliest offset
+ */
+static void state_set_offset_earliest(sub_test_state_t *state) {
+ const char *cfg[] = {"share.auto.offset.reset", "SET", "earliest"};
+ test_IncrementalAlterConfigs_simple(
+ test_share_consumer_get_rk(state->consumers[0]),
+ RD_KAFKA_RESOURCE_GROUP, state->group_name, cfg, 1);
+}
+
+/**
+ * @brief Create a new topic with auto-generated name
+ */
+static const char *state_create_topic(sub_test_state_t *state,
+ rd_bool_t wait_exists) {
+ char name[128];
+
+ TEST_ASSERT(state->all_topic_cnt < MAX_TOPICS,
+ "Too many topics created");
+
+ rd_snprintf(name, sizeof(name), "0170-t%d", state->all_topic_cnt);
+ state->all_topics[state->all_topic_cnt] =
+ rd_strdup(test_mk_topic_name(name, 1));
+
+ if (wait_exists) {
+ test_create_topic_wait_exists(
+ NULL, state->all_topics[state->all_topic_cnt], 1, -1,
+ 30000);
+ }
+
+ state->msgs_produced[state->all_topic_cnt] = 0;
+ return state->all_topics[state->all_topic_cnt++];
+}
+
+/**
+ * @brief Execute TEST_OP_SUBSCRIBE
+ */
+static void exec_subscribe(sub_test_state_t *state, const test_op_t *op) {
+ rd_kafka_topic_partition_list_t *tlist;
+ int cidx = op->consumer_idx;
+ int i, r;
+
+ TEST_SAY(" SUBSCRIBE: %d topic(s), repeat=%d, consumer=%d\n",
+ op->topic_cnt, op->repeat_cnt, cidx);
+
+ /* Save old subscription for RESUBSCRIBE verification */
+ state->old_sub_start_idx = state->sub_start_idx[cidx];
+ state->old_sub_count = state->sub_count[cidx];
+
+ /* Track new subscription */
+ state->sub_start_idx[cidx] = state->all_topic_cnt;
+ state->sub_count[cidx] = op->topic_cnt;
+
+ /* Create topics and build subscription list */
+ tlist = rd_kafka_topic_partition_list_new(op->topic_cnt);
+ for (i = 0; i < op->topic_cnt; i++) {
+ const char *topic = state_create_topic(
+ state, !(op->flags & TEST_OP_F_SKIP_TOPIC_CREATE));
+ rd_kafka_topic_partition_list_add(tlist, topic,
+ RD_KAFKA_PARTITION_UA);
+ }
+
+ /* Subscribe (possibly multiple times) */
+ for (r = 0; r < op->repeat_cnt; r++) {
+ TEST_CALL_ERR__(
+ rd_kafka_share_subscribe(state->consumers[cidx], tlist));
+ }
+
+ rd_kafka_topic_partition_list_destroy(tlist);
+}
+
+/**
+ * @brief Execute TEST_OP_SUBSCRIBE_ADD (incremental - add to existing
+ * subscription)
+ */
+static void exec_subscribe_add(sub_test_state_t *state, const test_op_t *op) {
+ rd_kafka_topic_partition_list_t *tlist;
+ int cidx = op->consumer_idx;
+ int i;
+ int new_start = state->all_topic_cnt;
+
+ TEST_SAY(
+ " SUBSCRIBE_ADD: adding %d topic(s) to existing %d, consumer=%d\n",
+ op->topic_cnt, state->sub_count[cidx], cidx);
+
+ /* Build subscription list including existing + new topics */
+ tlist = rd_kafka_topic_partition_list_new(state->sub_count[cidx] +
+ op->topic_cnt);
+
+ /* Add existing subscribed topics */
+ for (i = 0; i < state->sub_count[cidx]; i++) {
+ int idx = state->sub_start_idx[cidx] + i;
+ rd_kafka_topic_partition_list_add(tlist, state->all_topics[idx],
+ RD_KAFKA_PARTITION_UA);
+ }
+
+ /* Create and add new topics */
+ for (i = 0; i < op->topic_cnt; i++) {
+ const char *topic = state_create_topic(state, rd_true);
+ rd_kafka_topic_partition_list_add(tlist, topic,
+ RD_KAFKA_PARTITION_UA);
+ }
+
+ TEST_CALL_ERR__(
+ rd_kafka_share_subscribe(state->consumers[cidx], tlist));
+
+ /* Update subscription tracking - topics are now spread across ranges */
+ state->sub_start_idx[cidx] = new_start - state->sub_count[cidx];
+ state->sub_count[cidx] += op->topic_cnt;
+
+ rd_kafka_topic_partition_list_destroy(tlist);
+}
+
+/**
+ * @brief Execute TEST_OP_CREATE_TOPIC (create topics that weren't created)
+ */
+static void exec_create_topic(sub_test_state_t *state, const test_op_t *op) {
+ int cidx = op->consumer_idx;
+ int i;
+
+ TEST_SAY(" CREATE_TOPIC: creating subscribed topics for consumer=%d\n",
+ cidx);
+
+ /* Create the topics that were subscribed to but not yet created */
+ for (i = 0; i < state->sub_count[cidx]; i++) {
+ int idx = state->sub_start_idx[cidx] + i;
+ if (state->all_topics[idx]) {
+ test_create_topic_wait_exists(
+ NULL, state->all_topics[idx], 1, -1, 30000);
+ }
+ }
+}
+
+/**
+ * @brief Execute TEST_OP_SUBSCRIBE_EXISTING (subscribe to all created topics)
+ */
+static void exec_subscribe_existing(sub_test_state_t *state,
+ const test_op_t *op) {
+ rd_kafka_topic_partition_list_t *tlist;
+ int cidx = op->consumer_idx;
+ int i;
+
+ TEST_SAY(" SUBSCRIBE_EXISTING: %d topic(s), consumer=%d\n",
+ state->all_topic_cnt, cidx);
+
+ tlist = rd_kafka_topic_partition_list_new(state->all_topic_cnt);
+
+ for (i = 0; i < state->all_topic_cnt; i++) {
+ rd_kafka_topic_partition_list_add(tlist, state->all_topics[i],
+ RD_KAFKA_PARTITION_UA);
+ }
+
+ TEST_CALL_ERR__(
+ rd_kafka_share_subscribe(state->consumers[cidx], tlist));
+
+ state->sub_start_idx[cidx] = 0;
+ state->sub_count[cidx] = state->all_topic_cnt;
+
+ rd_kafka_topic_partition_list_destroy(tlist);
+}
+
+/**
+ * @brief Execute TEST_OP_PRODUCE_TO_TOPIC (produce to specific topic by index)
+ */
+static void exec_produce_to_topic(sub_test_state_t *state,
+ const test_op_t *op) {
+ int cidx = op->consumer_idx;
+ int idx = state->sub_start_idx[cidx] + op->topic_idx;
+
+ TEST_ASSERT(op->topic_idx < state->sub_count[cidx],
+ "Topic index %d out of range (sub_count=%d)", op->topic_idx,
+ state->sub_count[cidx]);
+
+ TEST_SAY(" PRODUCE_TO_TOPIC: %d msgs to topic[%d] (%s)\n",
+ op->msgs_per_topic, op->topic_idx, state->all_topics[idx]);
+
+ test_produce_msgs_easy(state->all_topics[idx], 0, 0,
+ op->msgs_per_topic);
+ state->msgs_produced[idx] += op->msgs_per_topic;
+}
+
+/**
+ * @brief Execute TEST_OP_RESUBSCRIBE (replace subscription with new topics)
+ */
+static void exec_resubscribe(sub_test_state_t *state, const test_op_t *op) {
+ rd_kafka_topic_partition_list_t *tlist;
+ int cidx = op->consumer_idx;
+ int i;
+
+ TEST_SAY(" RESUBSCRIBE: %d new topic(s), consumer=%d\n", op->topic_cnt,
+ cidx);
+
+ /* Save old subscription */
+ state->old_sub_start_idx = state->sub_start_idx[cidx];
+ state->old_sub_count = state->sub_count[cidx];
+
+ /* Track new subscription */
+ state->sub_start_idx[cidx] = state->all_topic_cnt;
+ state->sub_count[cidx] = op->topic_cnt;
+
+ /* Create new topics */
+ tlist = rd_kafka_topic_partition_list_new(op->topic_cnt);
+ for (i = 0; i < op->topic_cnt; i++) {
+ const char *topic = state_create_topic(state, rd_true);
+ rd_kafka_topic_partition_list_add(tlist, topic,
+ RD_KAFKA_PARTITION_UA);
+ }
+
+ TEST_CALL_ERR__(
+ rd_kafka_share_subscribe(state->consumers[cidx], tlist));
+ rd_kafka_topic_partition_list_destroy(tlist);
+}
+
+/**
+ * @brief Execute TEST_OP_UNSUBSCRIBE
+ */
+static void exec_unsubscribe(sub_test_state_t *state, const test_op_t *op) {
+ int cidx = op->consumer_idx;
+ int r;
+
+ TEST_SAY(" UNSUBSCRIBE: repeat=%d, consumer=%d\n", op->repeat_cnt,
+ cidx);
+
+ for (r = 0; r < op->repeat_cnt; r++) {
+ TEST_CALL_ERR__(
+ rd_kafka_share_unsubscribe(state->consumers[cidx]));
+ }
+
+ state->sub_count[cidx] = 0;
+}
+
+/**
+ * @brief Execute TEST_OP_PRODUCE
+ */
+static void exec_produce(sub_test_state_t *state, const test_op_t *op) {
+ int cidx = op->consumer_idx;
+ int start_idx, count, i;
+
+ if (op->flags & TEST_OP_F_PRODUCE_TO_OLD) {
+ start_idx = state->old_sub_start_idx;
+ count = state->old_sub_count;
+ TEST_SAY(" PRODUCE: %d msgs/topic to OLD %d topic(s)\n",
+ op->msgs_per_topic, count);
+ } else {
+ start_idx = state->sub_start_idx[cidx];
+ count = state->sub_count[cidx];
+ TEST_SAY(" PRODUCE: %d msgs/topic to %d topic(s)\n",
+ op->msgs_per_topic, count);
+ }
+
+ for (i = 0; i < count; i++) {
+ int idx = start_idx + i;
+ test_produce_msgs_easy(state->all_topics[idx], 0, 0,
+ op->msgs_per_topic);
+ state->msgs_produced[idx] += op->msgs_per_topic;
+ }
+}
+
+/**
+ * @brief Execute TEST_OP_CONSUME
+ */
+static void exec_consume(sub_test_state_t *state, const test_op_t *op) {
+ int cidx = op->consumer_idx;
+ int start_idx = state->sub_start_idx[cidx];
+ int count = state->sub_count[cidx];
+ const char *topics[MAX_TOPICS];
+ int i, consumed;
+
+ /* Build expected topics array */
+ for (i = 0; i < count; i++) {
+ topics[i] = state->all_topics[start_idx + i];
+ }
+
+ if (op->expected_msgs >= 0) {
+ TEST_SAY(" CONSUME: expecting %d msgs from %d topic(s)\n",
+ op->expected_msgs, count);
+ consumed = test_share_consume_msgs(
+ state->consumers[cidx], op->expected_msgs, 25, 3000,
+ count > 0 ? topics : NULL, count);
+
+ if (op->flags & TEST_OP_F_VERIFY_NO_OLD_MSGS) {
+ TEST_ASSERT(consumed >= 0,
+ "Received message from old subscription!");
+ }
+ TEST_ASSERT(consumed == op->expected_msgs,
+ "Expected %d messages, got %d", op->expected_msgs,
+ consumed);
+ } else {
+ /* Consume any available */
+ TEST_SAY(" CONSUME: any available from %d topic(s)\n", count);
+ test_share_consume_msgs(state->consumers[cidx], 100, 10, 2000,
+ count > 0 ? topics : NULL, count);
+ }
+}
+
+/**
+ * @brief Execute TEST_OP_VERIFY_SUB_CNT
+ */
+static void exec_verify_sub_cnt(sub_test_state_t *state, const test_op_t *op) {
+ int cidx = op->consumer_idx;
+ rd_kafka_topic_partition_list_t *sub;
+
+ TEST_SAY(" VERIFY_SUB_CNT: expecting %d, consumer=%d\n",
+ op->expected_sub_cnt, cidx);
+
+ sub = test_get_subscription(state->consumers[cidx]);
+ TEST_ASSERT(sub->cnt == op->expected_sub_cnt,
+ "Expected %d subscriptions, got %d", op->expected_sub_cnt,
+ sub->cnt);
+ rd_kafka_topic_partition_list_destroy(sub);
+}
+
+/**
+ * @brief Execute TEST_OP_DELETE_TOPIC
+ */
+static void exec_delete_topic(sub_test_state_t *state, const test_op_t *op) {
+ int cidx = op->consumer_idx;
+ int idx = state->sub_start_idx[cidx] + op->topic_idx;
+
+ TEST_ASSERT(op->topic_idx < state->sub_count[cidx],
+ "Topic index %d out of range (sub_count=%d)", op->topic_idx,
+ state->sub_count[cidx]);
+
+ TEST_SAY(" DELETE_TOPIC: index %d (%s)\n", op->topic_idx,
+ state->all_topics[idx]);
+
+ test_delete_topic(test_share_consumer_get_rk(state->consumers[0]),
+ state->all_topics[idx]);
+
+ /* Mark as deleted to skip during cleanup */
+ state->topic_deleted[idx] = rd_true;
+}
+
+/**
+ * @brief Execute TEST_OP_WAIT
+ */
+static void exec_wait(sub_test_state_t *state, const test_op_t *op) {
+ TEST_SAY(" WAIT: %d ms\n", op->wait_ms);
+ rd_sleep(op->wait_ms / 1000);
+ if (op->wait_ms % 1000)
+ rd_usleep((op->wait_ms % 1000) * 1000, NULL);
+}
+
+/**
+ * @brief Execute TEST_OP_CREATE_CONSUMER
+ */
+static void exec_create_consumer(sub_test_state_t *state, const test_op_t *op) {
+ int cidx = op->consumer_idx;
+
+ TEST_SAY(" CREATE_CONSUMER: index %d\n", cidx);
+
+ TEST_ASSERT(cidx < MAX_CONSUMERS, "Consumer index out of range");
+ TEST_ASSERT(state->consumers[cidx] == NULL,
+ "Consumer %d already exists", cidx);
+
+ state->consumers[cidx] = test_create_share_consumer(state->group_name);
+ if (cidx >= state->consumer_cnt)
+ state->consumer_cnt = cidx + 1;
+}
+
+/**
+ * @brief Execute TEST_OP_POLL_NO_SUB
+ */
+static void exec_poll_no_sub(sub_test_state_t *state, const test_op_t *op) {
+ rd_kafka_message_t *batch[TEST_SHARE_BATCH_SIZE];
+ rd_kafka_error_t *err;
+ size_t rcvd = 0;
+ int cidx = op->consumer_idx;
+
+ TEST_SAY(" POLL_NO_SUB: consumer=%d\n", cidx);
+
+ err = rd_kafka_share_consume_batch(state->consumers[cidx], 2000, batch,
+ &rcvd);
+ /* TODO KIP-932: Should return error */
+ if (err)
+ rd_kafka_error_destroy(err);
+}
+
+/**
+ * @brief Initialize test state
+ */
+static void state_init(sub_test_state_t *state,
+ const test_scenario_t *scenario) {
+ int i;
+
+ memset(state, 0, sizeof(*state));
+
+ rd_snprintf(state->group_name, sizeof(state->group_name), "share-%s",
+ scenario->name);
+
+ state->consumer_cnt =
+ scenario->consumer_cnt > 0 ? scenario->consumer_cnt : 1;
+
+ /* Create initial consumers */
+ for (i = 0; i < state->consumer_cnt; i++) {
+ state->consumers[i] =
+ test_create_share_consumer(state->group_name);
+ }
+
+ /* Set group offset to earliest */
+ state_set_offset_earliest(state);
+}
+
+/**
+ * @brief Cleanup test state
+ */
+static void state_cleanup(sub_test_state_t *state) {
+ int i;
+
+ /* Delete all created topics (skip already deleted ones) */
+ for (i = 0; i < state->all_topic_cnt; i++) {
+ if (state->all_topics[i] && !state->topic_deleted[i]) {
+ test_delete_topic(
+ test_share_consumer_get_rk(state->consumers[0]),
+ state->all_topics[i]);
+ rd_free(state->all_topics[i]);
+ }
+ }
+
+ /* Destroy all consumers */
+ for (i = 0; i < MAX_CONSUMERS; i++) {
+ if (state->consumers[i]) {
+ rd_kafka_share_consumer_close(state->consumers[i]);
+ rd_kafka_share_destroy(state->consumers[i]);
+ }
+ }
+}
+
+/**
+ * @brief Run a test scenario
+ */
+static void do_test_scenario(const test_scenario_t *scenario) {
+ sub_test_state_t state;
+ int op_idx;
+
+ TEST_SAY("\n");
+ TEST_SAY(
+ "============================================================\n");
+ TEST_SAY("=== %s ===\n", scenario->name);
+ TEST_SAY(
+ "============================================================\n");
+
+ state_init(&state, scenario);
+
+ /* Execute operations */
+ for (op_idx = 0; scenario->ops[op_idx].op != TEST_OP_END; op_idx++) {
+ const test_op_t *op = &scenario->ops[op_idx];
+
+ switch (op->op) {
+ case TEST_OP_SUBSCRIBE:
+ exec_subscribe(&state, op);
+ break;
+ case TEST_OP_SUBSCRIBE_ADD:
+ exec_subscribe_add(&state, op);
+ break;
+ case TEST_OP_RESUBSCRIBE:
+ exec_resubscribe(&state, op);
+ break;
+ case TEST_OP_UNSUBSCRIBE:
+ exec_unsubscribe(&state, op);
+ break;
+ case TEST_OP_PRODUCE:
+ exec_produce(&state, op);
+ break;
+ case TEST_OP_PRODUCE_TO_TOPIC:
+ exec_produce_to_topic(&state, op);
+ break;
+ case TEST_OP_CONSUME:
+ exec_consume(&state, op);
+ break;
+ case TEST_OP_VERIFY_SUB_CNT:
+ exec_verify_sub_cnt(&state, op);
+ break;
+ case TEST_OP_DELETE_TOPIC:
+ exec_delete_topic(&state, op);
+ break;
+ case TEST_OP_WAIT:
+ exec_wait(&state, op);
+ break;
+ case TEST_OP_CREATE_CONSUMER:
+ exec_create_consumer(&state, op);
+ break;
+ case TEST_OP_CREATE_TOPIC:
+ exec_create_topic(&state, op);
+ break;
+ case TEST_OP_SUBSCRIBE_EXISTING:
+ exec_subscribe_existing(&state, op);
+ break;
+ case TEST_OP_POLL_NO_SUB:
+ exec_poll_no_sub(&state, op);
+ break;
+ default:
+ TEST_FAIL("Unknown operation: %d", op->op);
+ }
+ }
+
+ state_cleanup(&state);
+
+ TEST_SAY("=== %s: PASSED ===\n", scenario->name);
+}
+
+
+/**
+ * Basic subscription tests
+ */
+static const test_scenario_t test_single_subscribe = {
+ .name = "single-subscribe",
+ .ops = {SUBSCRIBE(2), PRODUCE(5), VERIFY_SUB(2), CONSUME(10),
+ TEST_OPS_END()}};
+
+static const test_scenario_t test_single_unsubscribe = {
+ .name = "single-unsubscribe",
+ .ops = {SUBSCRIBE(2), PRODUCE(5), CONSUME(10), UNSUBSCRIBE(), VERIFY_SUB(0),
+ TEST_OPS_END()}};
+
+static const test_scenario_t test_repeated_subscribe = {
+ .name = "repeated-subscribe-no-duplicates",
+ .ops = {SUBSCRIBE_REPEAT(2, 3), /* Subscribe 3 times to same topics */
+ PRODUCE(5), VERIFY_SUB(2), /* Should still be 2, not 6 */
+ CONSUME(10), TEST_OPS_END()}};
+
+static const test_scenario_t test_repeated_unsubscribe = {
+ .name = "repeated-unsubscribe-no-error",
+ .ops = {SUBSCRIBE(2), PRODUCE(5), CONSUME(10),
+ UNSUBSCRIBE_REPEAT(3), /* Unsubscribe 3 times */
+ VERIFY_SUB(0), TEST_OPS_END()}};
+
+/**
+ * Subscription replacement tests
+ */
+static const test_scenario_t test_topic_switch = {
+ .name = "topic-switch",
+ .ops = {SUBSCRIBE(2), PRODUCE(10), CONSUME_ANY(),
+ RESUBSCRIBE(2), /* Switch to 2 new topics */
+ PRODUCE(10), /* Produce to new topics */
+ PRODUCE_TO_OLD(5), /* Produce to old topics */
+ CONSUME_VERIFY_NO_OLD(20), /* Should only get new topic msgs */
+ TEST_OPS_END()}};
+
+static const test_scenario_t test_incremental_subscription = {
+ .name = "incremental-subscription",
+ .ops = {/* Start with 1 topic */
+ SUBSCRIBE(1), PRODUCE(10), VERIFY_SUB(1), CONSUME(10),
+ /* Add 1 more topic (now 2 total) */
+ SUBSCRIBE_ADD(1), PRODUCE(10), VERIFY_SUB(2),
+ CONSUME(20), /* 10 from each of 2 topics */
+ /* Add 1 more topic (now 3 total) */
+ SUBSCRIBE_ADD(1), PRODUCE(10), VERIFY_SUB(3),
+ CONSUME(30), /* 10 from each of 3 topics */
+ TEST_OPS_END()}};
+
+/**
+ * Edge case tests
+ */
+static const test_scenario_t test_subscribe_before_topic_exists = {
+ .name = "subscribe-before-topic-exists",
+ .ops = {SUBSCRIBE_NO_CREATE(1), /* Subscribe without creating topic */
+ CREATE_TOPIC(0), /* Now create the subscribed topic */
+ PRODUCE(5), /* Produce to the topic */
+ CONSUME(5), /* Should receive all messages */
+ TEST_OPS_END()}};
+
+static const test_scenario_t test_poll_empty_topic = {
+ .name = "poll-empty-topic",
+ .ops = {SUBSCRIBE(1),
+ /* Don't produce - topic is empty */
+ CONSUME(0), /* Should return 0, not error */
+ TEST_OPS_END()}};
+
+static const test_scenario_t test_poll_no_subscription = {
+ .name = "poll-no-subscription",
+ .ops = {POLL_NO_SUB(), /* Poll without subscribing */
+ TEST_OPS_END()}};
+
+static const test_scenario_t test_poll_after_unsubscribe = {
+ .name = "poll-after-unsubscribe",
+ .ops = {SUBSCRIBE(1), PRODUCE(5), CONSUME_ANY(), /* Consume some */
+ UNSUBSCRIBE(), POLL_NO_SUB(), /* Poll after unsubscribe */
+ TEST_OPS_END()}};
+
+/**
+ * Topic deletion tests
+ */
+static const test_scenario_t test_topic_deletion = {
+ .name = "topic-deletion-while-subscribed",
+ .ops = {SUBSCRIBE(2), PRODUCE(10), CONSUME_ANY(),
+ DELETE_TOPIC(1), /* Delete second topic */
+ WAIT_MS(3000),
+ PRODUCE_TO_TOPIC(0, 5), /* Produce to remaining topic */
+ CONSUME_ANY(), /* Continue consuming from remaining */
+ TEST_OPS_END()}};
+
+/**
+ * Stress tests
+ */
+static const test_scenario_t test_rapid_updates = {
+ .name = "rapid-subscription-updates",
+ .ops = {SUBSCRIBE(2), RESUBSCRIBE(1), RESUBSCRIBE(3), UNSUBSCRIBE(),
+ SUBSCRIBE(2), RESUBSCRIBE(2), UNSUBSCRIBE(), SUBSCRIBE(3),
+ VERIFY_SUB(3), TEST_OPS_END()}};
+
+/**
+ * @brief Multi-consumer overlap test (standalone)
+ *
+ * Two consumers in same group with overlapping subscriptions:
+ * - Consumer 0: [shared, c0_only]
+ * - Consumer 1: [shared, c1_only]
+ *
+ * This test verifies share group consumers can have overlapping
+ * subscriptions and both receive messages from shared topics.
+ */
+static void do_test_multi_consumer_overlap(void) {
+ const char *group = test_mk_topic_name("share-overlap", 1);
+ char *shared = rd_strdup(test_mk_topic_name("0170-shared", 1));
+ char *c0_only = rd_strdup(test_mk_topic_name("0170-c0only", 1));
+ char *c1_only = rd_strdup(test_mk_topic_name("0170-c1only", 1));
+ const char *c0_topics[] = {shared, c0_only};
+ const char *c1_topics[] = {shared, c1_only};
+ rd_kafka_share_t *rkshare0, *rkshare1;
+ int c0_cnt = 0, c1_cnt = 0;
+ int attempts;
+ const char *cfg[] = {"share.auto.offset.reset", "SET", "earliest"};
+
+ TEST_SAY("\n");
+ TEST_SAY(
+ "============================================================\n");
+ TEST_SAY("=== multi-consumer-overlapping-subscriptions ===\n");
+ TEST_SAY(
+ "============================================================\n");
+
+ /* Create topics */
+ test_create_topic_wait_exists(NULL, shared, 1, -1, 30000);
+ test_create_topic_wait_exists(NULL, c0_only, 1, -1, 30000);
+ test_create_topic_wait_exists(NULL, c1_only, 1, -1, 30000);
+
+ /* Produce messages */
+ test_produce_msgs_easy(shared, 0, 0, 20);
+ test_produce_msgs_easy(c0_only, 0, 0, 10);
+ test_produce_msgs_easy(c1_only, 0, 0, 10);
+
+ /* Create consumers */
+ rkshare0 = test_create_share_consumer(group);
+ rkshare1 = test_create_share_consumer(group);
+
+ /* Set group offset */
+ test_IncrementalAlterConfigs_simple(
+ test_share_consumer_get_rk(rkshare0), RD_KAFKA_RESOURCE_GROUP,
+ group, cfg, 1);
+
+ /* Subscribe with overlapping topics */
+ test_share_consumer_subscribe_multi(rkshare0, 2, shared, c0_only);
+ test_share_consumer_subscribe_multi(rkshare1, 2, shared, c1_only);
+
+ /* Consume - alternate between consumers */
+ attempts = 20;
+ while ((c0_cnt + c1_cnt) < 10 && attempts-- > 0) {
+ int batch_cnt = 0;
+ int ret;
+
+ ret = test_share_consume_batch(rkshare0, 2000, c0_topics, 2,
+ &batch_cnt);
+ TEST_ASSERT(ret >= 0, "C0 wrong topic");
+ c0_cnt += batch_cnt;
+
+ batch_cnt = 0;
+ ret = test_share_consume_batch(rkshare1, 2000, c1_topics, 2,
+ &batch_cnt);
+ TEST_ASSERT(ret >= 0, "C1 wrong topic");
+ c1_cnt += batch_cnt;
+ }
+
+ TEST_SAY("C0: %d, C1: %d (total: %d)\n", c0_cnt, c1_cnt,
+ c0_cnt + c1_cnt);
+ TEST_ASSERT(c0_cnt > 0 || c1_cnt > 0, "no messages received");
+
+ /* Cleanup */
+ rd_kafka_share_consumer_close(rkshare0);
+ rd_kafka_share_consumer_close(rkshare1);
+ rd_kafka_share_destroy(rkshare0);
+ rd_kafka_share_destroy(rkshare1);
+
+ rd_free(shared);
+ rd_free(c0_only);
+ rd_free(c1_only);
+
+ TEST_SAY("=== multi-consumer-overlapping-subscriptions: PASSED ===\n");
+}
+
+
+int main_0170_share_consumer_subscription(int argc, char **argv) {
+
+ /* Basic subscription tests */
+ do_test_scenario(&test_single_subscribe);
+ do_test_scenario(&test_single_unsubscribe);
+ do_test_scenario(&test_repeated_subscribe);
+ do_test_scenario(&test_repeated_unsubscribe);
+
+ /* Subscription replacement tests */
+ do_test_scenario(&test_topic_switch);
+ do_test_scenario(&test_incremental_subscription);
+
+ /* Edge case tests */
+ do_test_scenario(&test_subscribe_before_topic_exists);
+ do_test_scenario(&test_poll_empty_topic);
+ do_test_scenario(&test_poll_no_subscription);
+ do_test_scenario(&test_poll_after_unsubscribe);
+
+ /* Topic deletion tests */
+ do_test_scenario(&test_topic_deletion);
+
+ /* Stress tests */
+ do_test_scenario(&test_rapid_updates);
+
+ /* Multi-consumer tests (standalone - requires shared topics) */
+ do_test_multi_consumer_overlap();
+
+ return 0;
+}
diff --git a/tests/0171-share_consumer_consume.c b/tests/0171-share_consumer_consume.c
new file mode 100644
index 0000000000..db4f621d5a
--- /dev/null
+++ b/tests/0171-share_consumer_consume.c
@@ -0,0 +1,498 @@
+/*
+ * 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 Maximum supported values for test configuration
+ */
+#define MAX_CONSUMERS 16
+#define MAX_TOPICS 16
+#define MAX_PARTITIONS 32
+#define BATCH_SIZE 10000
+
+/**
+ * @brief Test configuration structure
+ *
+ * This structure defines all parameters for a share consumer test scenario.
+ *
+ * Example configurations:
+ *
+ * 1. Multiple consumers, single topic, multiple partitions:
+ * { .consumer_cnt = 2, .topic_cnt = 1, .partitions = {3},
+ * .msgs_per_partition = 100, .group_name = "test-group" }
+ *
+ * 2. Single consumer, multiple topics, single partition each:
+ * { .consumer_cnt = 1, .topic_cnt = 2, .partitions = {1, 1},
+ * .msgs_per_partition = 100, .group_name = "test-group" }
+ *
+ * 3. Multiple consumers, multiple topics, multiple partitions:
+ * { .consumer_cnt = 3, .topic_cnt = 2, .partitions = {3, 2},
+ * .msgs_per_partition = 50, .group_name = "test-group" }
+ */
+typedef struct {
+ int consumer_cnt; /**< Number of consumers to create */
+ int topic_cnt; /**< Number of topics to create */
+ int partitions[MAX_TOPICS]; /**< Partition count for each topic */
+ int msgs_per_partition; /**< Messages to produce per partition */
+ const char *group_name; /**< Share group name */
+ const char *test_name; /**< Test description for logging */
+ int poll_timeout_ms; /**< Timeout for each poll (default: 3000) */
+ int max_attempts; /**< Max poll attempts (default: auto-calculated) */
+} share_test_config_t;
+
+/**
+ * @brief Test state/results structure
+ */
+typedef struct {
+ rd_kafka_share_t *consumers[MAX_CONSUMERS];
+ char *topic_names[MAX_TOPICS];
+ int per_consumer_count[MAX_CONSUMERS];
+ int total_consumed;
+ int total_expected;
+} share_test_state_t;
+
+/**
+ * @brief Create share consumers
+ */
+static void create_share_consumers(share_test_config_t *config,
+ share_test_state_t *state) {
+ int i;
+
+ for (i = 0; i < config->consumer_cnt; i++) {
+ state->consumers[i] =
+ test_create_share_consumer(config->group_name);
+ }
+
+ TEST_SAY("Created %d share consumer(s)\n", config->consumer_cnt);
+}
+
+/**
+ * @brief Create topics and produce messages
+ */
+static void setup_topics_and_produce(share_test_config_t *config,
+ share_test_state_t *state) {
+ int t, p;
+ int total_partitions = 0;
+
+ state->total_expected = 0;
+
+ for (t = 0; t < config->topic_cnt; t++) {
+ /* Generate unique topic name */
+ state->topic_names[t] =
+ rd_strdup(test_mk_topic_name("0171-share-test", 1));
+
+ /* Create topic with specified partitions */
+ test_create_topic_wait_exists(NULL, state->topic_names[t],
+ config->partitions[t], -1,
+ 60 * 1000);
+
+ /* Produce messages to each partition */
+ for (p = 0; p < config->partitions[t]; p++) {
+ test_produce_msgs_easy(state->topic_names[t], p, p,
+ config->msgs_per_partition);
+ state->total_expected += config->msgs_per_partition;
+ }
+
+ total_partitions += config->partitions[t];
+ TEST_SAY("Topic '%s': %d partition(s), %d msgs/partition\n",
+ state->topic_names[t], config->partitions[t],
+ config->msgs_per_partition);
+ }
+
+ TEST_SAY(
+ "Setup complete: %d topic(s), %d total partition(s), "
+ "%d total messages\n",
+ config->topic_cnt, total_partitions, state->total_expected);
+}
+
+/**
+ * @brief Subscribe consumers to topics
+ */
+static void subscribe_consumers(share_test_config_t *config,
+ share_test_state_t *state) {
+ rd_kafka_topic_partition_list_t *subs;
+ const char *grp_conf[] = {"share.auto.offset.reset", "SET", "earliest"};
+ int t, i;
+
+ /* Set group config using first consumer */
+ test_IncrementalAlterConfigs_simple(
+ test_share_consumer_get_rk(state->consumers[0]),
+ RD_KAFKA_RESOURCE_GROUP, config->group_name, grp_conf, 1);
+
+ /* Build subscription list */
+ subs = rd_kafka_topic_partition_list_new(config->topic_cnt);
+ for (t = 0; t < config->topic_cnt; t++) {
+ rd_kafka_topic_partition_list_add(subs, state->topic_names[t],
+ RD_KAFKA_PARTITION_UA);
+ }
+
+ /* Subscribe all consumers */
+ for (i = 0; i < config->consumer_cnt; i++) {
+ rd_kafka_share_subscribe(state->consumers[i], subs);
+ }
+
+ rd_kafka_topic_partition_list_destroy(subs);
+
+ TEST_SAY("Subscribed %d consumer(s) to %d topic(s)\n",
+ config->consumer_cnt, config->topic_cnt);
+}
+
+/**
+ * @brief Consume messages function
+ */
+static void consume_messages(share_test_config_t *config,
+ share_test_state_t *state) {
+ rd_kafka_message_t *batch[BATCH_SIZE];
+ int attempts;
+ int poll_timeout;
+ int i;
+
+ /* Initialize per-consumer counts */
+ for (i = 0; i < config->consumer_cnt; i++) {
+ state->per_consumer_count[i] = 0;
+ }
+ state->total_consumed = 0;
+
+ /* Calculate defaults if not specified */
+ poll_timeout =
+ config->poll_timeout_ms > 0 ? config->poll_timeout_ms : 3000;
+
+ /* Auto-calculate attempts based on expected messages and consumers */
+ if (config->max_attempts > 0) {
+ attempts = config->max_attempts;
+ } else {
+ /* Heuristic: at least 10 attempts, or based on message count */
+ attempts = (state->total_expected / (BATCH_SIZE / 2)) + 10;
+ if (attempts < 20)
+ attempts = 20;
+ }
+
+ TEST_SAY(
+ "Starting consumption: expecting %d messages, "
+ "max %d attempts, %dms timeout\n",
+ state->total_expected, attempts, poll_timeout);
+
+ while (state->total_consumed < state->total_expected &&
+ attempts-- > 0) {
+ /* Round-robin poll across all consumers */
+ for (i = 0; i < config->consumer_cnt; i++) {
+ size_t rcvd = 0;
+ size_t m;
+ rd_kafka_error_t *err;
+
+ err = rd_kafka_share_consume_batch(
+ state->consumers[i], poll_timeout, batch, &rcvd);
+
+ if (err) {
+ rd_kafka_error_destroy(err);
+ continue;
+ }
+
+ for (m = 0; m < rcvd; m++) {
+ if (!batch[m]->err) {
+ state->per_consumer_count[i]++;
+ state->total_consumed++;
+ }
+ rd_kafka_message_destroy(batch[m]);
+ }
+
+ /* Early exit if we've consumed everything */
+ if (state->total_consumed >= state->total_expected)
+ break;
+ }
+
+ /* Progress logging */
+ if (config->consumer_cnt == 1) {
+ TEST_SAY("Progress: %d/%d\n", state->total_consumed,
+ state->total_expected);
+ } else {
+ /* Build distribution string for multiple consumers */
+ char dist[256] = {0};
+ int pos = 0;
+ for (i = 0; i < config->consumer_cnt && pos < 250;
+ i++) {
+ pos += rd_snprintf(
+ dist + pos, sizeof(dist) - pos, "c%d=%d ",
+ i, state->per_consumer_count[i]);
+ }
+ TEST_SAY("Progress: %d/%d (%s)\n",
+ state->total_consumed, state->total_expected,
+ dist);
+ }
+ }
+}
+
+/**
+ * @brief Cleanup function
+ */
+static void cleanup_test(share_test_config_t *config,
+ share_test_state_t *state) {
+ int t, i;
+
+ /* Delete topics using first consumer */
+ for (t = 0; t < config->topic_cnt; t++) {
+ if (state->topic_names[t]) {
+ test_delete_topic(
+ test_share_consumer_get_rk(state->consumers[0]),
+ state->topic_names[t]);
+ rd_free(state->topic_names[t]);
+ state->topic_names[t] = NULL;
+ }
+ }
+
+ /* Destroy consumers */
+ for (i = 0; i < config->consumer_cnt; i++) {
+ if (state->consumers[i]) {
+ rd_kafka_share_consumer_close(state->consumers[i]);
+ rd_kafka_share_destroy(state->consumers[i]);
+ state->consumers[i] = NULL;
+ }
+ }
+
+ TEST_SAY("Cleanup complete\n");
+}
+
+/**
+ * @brief Test Runner function
+ *
+ * This function handles:
+ * 1. Create consumers
+ * 2. Create topics and produce messages
+ * 3. Subscribe consumers
+ * 4. Consume and verify messages
+ * 5. Cleanup
+ *
+ * @param config Test configuration
+ * @returns 0 on success, -1 on failure
+ */
+static int run_share_consumer_test(share_test_config_t *config) {
+ share_test_state_t state = {0};
+ int i;
+ char dist_str[512] = {0};
+ int pos = 0;
+
+ /* Validate config */
+ TEST_ASSERT(config->consumer_cnt > 0 &&
+ config->consumer_cnt <= MAX_CONSUMERS,
+ "consumer_cnt must be 1-%d", MAX_CONSUMERS);
+ TEST_ASSERT(config->topic_cnt > 0 && config->topic_cnt <= MAX_TOPICS,
+ "topic_cnt must be 1-%d", MAX_TOPICS);
+ TEST_ASSERT(config->group_name != NULL, "group_name is required");
+
+ /* Print test header */
+ TEST_SAY("\n");
+ TEST_SAY(
+ "============================================================\n");
+ TEST_SAY("=== %s ===\n",
+ config->test_name ? config->test_name : "Share Consumer Test");
+ TEST_SAY("=== Consumers: %d, Topics: %d, Partitions: [",
+ config->consumer_cnt, config->topic_cnt);
+ for (i = 0; i < config->topic_cnt; i++) {
+ TEST_SAY("%d%s", config->partitions[i],
+ i < config->topic_cnt - 1 ? ", " : "");
+ }
+ TEST_SAY("], Msgs/Partition: %d ===\n", config->msgs_per_partition);
+ TEST_SAY(
+ "============================================================\n");
+
+ /* Execute test phases */
+ create_share_consumers(config, &state);
+ setup_topics_and_produce(config, &state);
+ subscribe_consumers(config, &state);
+ consume_messages(config, &state);
+
+ /* Verify results */
+ TEST_ASSERT(state.total_consumed == state.total_expected,
+ "Expected %d messages, consumed %d", state.total_expected,
+ state.total_consumed);
+
+ /* Build distribution string */
+ for (i = 0; i < config->consumer_cnt && pos < 500; i++) {
+ pos += rd_snprintf(dist_str + pos, sizeof(dist_str) - pos,
+ "c%d=%d ", i, state.per_consumer_count[i]);
+ }
+
+ TEST_SAY("✓ SUCCESS: Consumed %d/%d messages (distribution: %s)\n",
+ state.total_consumed, state.total_expected, dist_str);
+
+ /* Cleanup */
+ /** TODO KIP-932: Uncomment it when all the cleanups are properly done.
+ */
+ // cleanup_test(config, &state);
+
+ return 0;
+}
+
+
+/**
+ * @brief Single consumer, single topic, single partition
+ */
+static void test_single_consumer_single_topic_single_partition(void) {
+ share_test_config_t config = {
+ .consumer_cnt = 1,
+ .topic_cnt = 1,
+ .partitions = {1},
+ .msgs_per_partition = 1000,
+ .group_name = "share-1c-1t-1p",
+ .test_name = "Single consumer, single topic, single partition"};
+ run_share_consumer_test(&config);
+}
+
+/**
+ * @brief Single consumer, single topic, multiple partitions
+ */
+static void test_single_consumer_single_topic_multiple_partitions(void) {
+ share_test_config_t config = {
+ .consumer_cnt = 1,
+ .topic_cnt = 1,
+ .partitions = {3},
+ .msgs_per_partition = 500,
+ .group_name = "share-1c-1t-3p",
+ .test_name = "Single consumer, single topic, 3 partitions"};
+ run_share_consumer_test(&config);
+}
+
+/**
+ * @brief Single consumer, multiple topics, single partition each
+ */
+static void test_single_consumer_multiple_topic_single_partition(void) {
+ share_test_config_t config = {
+ .consumer_cnt = 1,
+ .topic_cnt = 2,
+ .partitions = {1, 1},
+ .msgs_per_partition = 500,
+ .group_name = "share-1c-2t-1p",
+ .test_name = "Single consumer, 2 topics, 1 partition each"};
+ run_share_consumer_test(&config);
+}
+
+/**
+ * @brief Single consumer, multiple topics, multiple partitions
+ */
+static void test_single_consumer_multiple_topic_multiple_partitions(void) {
+ share_test_config_t config = {
+ .consumer_cnt = 1,
+ .topic_cnt = 3,
+ .partitions = {2, 2, 2},
+ .msgs_per_partition = 500,
+ .group_name = "share-1c-3t-2p",
+ .test_name = "Single consumer, 3 topics, 2 partitions each"};
+ run_share_consumer_test(&config);
+}
+
+/**
+ * @brief Multiple consumers, single topic, single partition
+ */
+static void test_multiple_consumers_single_topic_single_partition(void) {
+ share_test_config_t config = {
+ .consumer_cnt = 2,
+ .topic_cnt = 1,
+ .partitions = {1},
+ .msgs_per_partition = 1000,
+ .group_name = "share-2c-1t-1p",
+ .test_name = "2 consumers, single topic, single partition"};
+ run_share_consumer_test(&config);
+}
+
+/**
+ * @brief Multiple consumers, single topic, multiple partitions
+ */
+static void test_multiple_consumers_single_topic_multiple_partitions(void) {
+ share_test_config_t config = {
+ .consumer_cnt = 2,
+ .topic_cnt = 1,
+ .partitions = {4},
+ .msgs_per_partition = 500,
+ .group_name = "share-2c-1t-4p",
+ .test_name = "2 consumers, single topic, 4 partitions"};
+ run_share_consumer_test(&config);
+}
+
+/**
+ * @brief Multiple consumers, multiple topics, multiple partitions
+ */
+static void test_multiple_consumers_multiple_topics_multiple_partitions(void) {
+ share_test_config_t config = {
+ .consumer_cnt = 3,
+ .topic_cnt = 2,
+ .partitions = {3, 3},
+ .msgs_per_partition = 500,
+ .group_name = "share-3c-2t-3p",
+ .test_name = "3 consumers, 2 topics, 3 partitions each"};
+ run_share_consumer_test(&config);
+}
+
+
+int main_0171_share_consumer_consume(int argc, char **argv) {
+
+ /* Single-consumer tests */
+ test_single_consumer_single_topic_single_partition(); /* Single
+ consumer,
+ single topic,
+ single
+ partition */
+ test_single_consumer_single_topic_multiple_partitions(); /* Single
+ consumer,
+ single
+ topic, multi
+ partitions
+ */
+ test_single_consumer_multiple_topic_single_partition(); /* Single
+ consumer,
+ multi topic,
+ single
+ partition
+ each */
+ test_single_consumer_multiple_topic_multiple_partitions(); /* Single
+ consumer,
+ multi
+ topics,
+ multi
+ partitions
+ each */
+
+ /* Multi-consumer tests */
+ test_multiple_consumers_single_topic_single_partition(); /* Multi
+ consumer
+ sharing
+ single
+ partition */
+ test_multiple_consumers_single_topic_multiple_partitions(); /* Multi
+ consumer,
+ multi
+ partition
+ */
+ test_multiple_consumers_multiple_topics_multiple_partitions(); /* Full
+ matrix:
+ multi
+ everything
+ */
+
+ return 0;
+}
diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt
index 324281bd99..18f4f10f5d 100644
--- a/tests/CMakeLists.txt
+++ b/tests/CMakeLists.txt
@@ -144,6 +144,10 @@ set(
0151-purge-brokers.c
0152-rebootstrap.c
0153-memberid.c
+ 0155-share_group_heartbeat_mock.c
+ 0156-kip932-sharefetch_mockbroker.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 51c6e2a293..d5bc5e2ba3 100644
--- a/tests/test.c
+++ b/tests/test.c
@@ -44,6 +44,7 @@
/* Typical include path would be , but this program
* is built from within the librdkafka source tree and thus differs. */
#include "rdkafka.h"
+#include "rdkafka_int.h"
int test_level = 2;
int test_seed = 0;
@@ -100,7 +101,7 @@ static const char *test_states[] = {
#define _TEST_DECL(NAME) extern int main_##NAME(int, char **)
#define _TEST(NAME, FLAGS, ...) \
- { .name = #NAME, .mainfunc = main_##NAME, .flags = FLAGS, __VA_ARGS__ }
+ {.name = #NAME, .mainfunc = main_##NAME, .flags = FLAGS, __VA_ARGS__}
/**
@@ -272,6 +273,10 @@ _TEST_DECL(0150_telemetry_mock);
_TEST_DECL(0151_purge_brokers_mock);
_TEST_DECL(0152_rebootstrap_local);
_TEST_DECL(0153_memberid);
+_TEST_DECL(0155_share_group_heartbeat_mock);
+_TEST_DECL(0156_kip932_sharefetch_mockbroker);
+_TEST_DECL(0170_share_consumer_subscription);
+_TEST_DECL(0171_share_consumer_consume);
/* Manual tests */
_TEST_DECL(8000_idle);
@@ -540,6 +545,11 @@ struct test tests[] = {
_TEST(0151_purge_brokers_mock, TEST_F_LOCAL),
_TEST(0152_rebootstrap_local, TEST_F_LOCAL),
_TEST(0153_memberid, TEST_F_LOCAL),
+ _TEST(0155_share_group_heartbeat_mock, TEST_F_LOCAL),
+ _TEST(0156_kip932_sharefetch_mockbroker, 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 */
_TEST(8000_idle, TEST_F_MANUAL),
@@ -2750,6 +2760,142 @@ rd_kafka_t *test_create_consumer(
return rk;
}
+/**
+ * @brief Create a share consumer with standard configuration.
+ *
+ * @param group_id The share group ID.
+ *
+ * @returns A new share consumer instance.
+ *
+ * @remark TODO: Remove explicit group.id and enable.auto.commit settings
+ * once these properties are added as defaults to
+ * rd_kafka_share_consumer_new().
+ */
+rd_kafka_share_t *test_create_share_consumer(const char *group_id) {
+ rd_kafka_share_t *rk;
+ rd_kafka_conf_t *conf;
+ char errstr[512];
+
+ test_conf_init(&conf, NULL, 60);
+
+ rd_kafka_conf_set(conf, "group.id", group_id, errstr, sizeof(errstr));
+ rd_kafka_conf_set(conf, "enable.auto.commit", "false", errstr,
+ sizeof(errstr));
+
+ rk = rd_kafka_share_consumer_new(conf, errstr, sizeof(errstr));
+ TEST_ASSERT(rk, "Failed to create share consumer: %s", errstr);
+
+ return rk;
+}
+
+
+/**
+ * @brief Consume share messages and verify they come from expected topics.
+ *
+ * @param rk Share consumer handle.
+ * @param timeout_ms Poll timeout in milliseconds.
+ * @param expected_topics Array of topic names that are valid sources
+ * (NULL to skip verification).
+ * @param expected_topic_cnt Number of topics in expected_topics array.
+ * @param out_valid Output: count of valid (non-error) messages received.
+ *
+ * @returns 0 on success, -1 if message from unexpected topic received.
+ */
+int test_share_consume_batch(rd_kafka_share_t *rk,
+ int timeout_ms,
+ const char **expected_topics,
+ int expected_topic_cnt,
+ int *out_valid) {
+ rd_kafka_message_t *batch[TEST_SHARE_BATCH_SIZE];
+ rd_kafka_error_t *err;
+ size_t rcvd = 0;
+ int valid = 0;
+ size_t i;
+ int j;
+ int ret = 0;
+
+ err = rd_kafka_share_consume_batch(rk, timeout_ms, batch, &rcvd);
+ if (err) {
+ rd_kafka_error_destroy(err);
+ *out_valid = 0;
+ return 0;
+ }
+
+ for (i = 0; i < rcvd; i++) {
+ if (batch[i]->err) {
+ /* Skip error messages */
+ rd_kafka_message_destroy(batch[i]);
+ continue;
+ }
+
+ if (expected_topics) {
+ const char *msg_topic =
+ rd_kafka_topic_name(batch[i]->rkt);
+ int found = 0;
+
+ /* Verify message is from expected topic */
+ for (j = 0; j < expected_topic_cnt; j++) {
+ if (strcmp(msg_topic, expected_topics[j]) ==
+ 0) {
+ found = 1;
+ break;
+ }
+ }
+
+ if (!found) {
+ TEST_SAY(
+ "ERROR: Received message from "
+ "unexpected topic '%s'\n",
+ msg_topic);
+ ret = -1;
+ }
+ }
+ valid++;
+ rd_kafka_message_destroy(batch[i]);
+ }
+
+ *out_valid = valid;
+ return ret;
+}
+
+
+/**
+ * @brief Consume share messages until expected count or max attempts.
+ *
+ * @param rk Share consumer handle.
+ * @param expected Number of messages to consume.
+ * @param max_attempts Maximum poll attempts.
+ * @param timeout_ms Timeout per poll in milliseconds.
+ * @param expected_topics Array of valid topic names (NULL to skip
+ * verification).
+ * @param expected_topic_cnt Number of topics in expected_topics.
+ *
+ * @returns Number of messages consumed, or -1 if message from wrong topic.
+ */
+int test_share_consume_msgs(rd_kafka_share_t *rk,
+ int expected,
+ int max_attempts,
+ int timeout_ms,
+ const char **expected_topics,
+ int expected_topic_cnt) {
+ int total = 0;
+
+ while (total < expected && max_attempts-- > 0) {
+ int batch_cnt = 0;
+ int ret;
+
+ ret = test_share_consume_batch(rk, timeout_ms, expected_topics,
+ expected_topic_cnt, &batch_cnt);
+ if (ret < 0)
+ return -1; /* Wrong topic detected */
+
+ total += batch_cnt;
+ }
+
+ return total;
+}
+
+
rd_kafka_topic_t *test_create_consumer_topic(rd_kafka_t *rk,
const char *topic) {
rd_kafka_topic_t *rkt;
@@ -3282,6 +3428,72 @@ void test_consumer_subscribe_multi(rd_kafka_t *rk, int topic_count, ...) {
}
+/**
+ * @brief Start subscribing for multiple topics (share consumers).
+ */
+void test_share_consumer_subscribe_multi(rd_kafka_share_t *rk,
+ int topic_count,
+ ...) {
+ rd_kafka_topic_partition_list_t *topics;
+ rd_kafka_resp_err_t err;
+ va_list ap;
+ int i;
+
+ topics = rd_kafka_topic_partition_list_new(topic_count);
+
+ va_start(ap, topic_count);
+ for (i = 0; i < topic_count; i++) {
+ const char *topic = va_arg(ap, const char *);
+ rd_kafka_topic_partition_list_add(topics, topic,
+ RD_KAFKA_PARTITION_UA);
+ }
+ va_end(ap);
+
+ err = rd_kafka_share_subscribe(rk, topics);
+ if (err)
+ TEST_FAIL("%s: Failed to subscribe to topics: %s\n",
+ rd_kafka_name(test_share_consumer_get_rk(rk)),
+ rd_kafka_err2str(err));
+
+ rd_kafka_topic_partition_list_destroy(topics);
+}
+
+
+/**
+ * @brief Get underlying rd_kafka_t handle from share consumer.
+ *
+ * @param rkshare Share consumer handle.
+ * @returns The underlying rd_kafka_t handle.
+ */
+rd_kafka_t *test_share_consumer_get_rk(rd_kafka_share_t *rkshare) {
+ return rkshare->rkshare_rk;
+}
+
+
+/**
+ * @brief Get current subscription list for share consumers.
+ *
+ * @returns The subscription list. Caller must destroy with
+ * rd_kafka_topic_partition_list_destroy().
+ */
+rd_kafka_topic_partition_list_t *test_get_subscription(rd_kafka_share_t *rk) {
+ rd_kafka_topic_partition_list_t *subscription = NULL;
+ rd_kafka_resp_err_t err;
+
+ err = rd_kafka_share_subscription(rk, &subscription);
+ if (err)
+ TEST_FAIL("%s: Failed to get subscription: %s\n",
+ rd_kafka_name(test_share_consumer_get_rk(rk)),
+ rd_kafka_err2str(err));
+
+ TEST_ASSERT(subscription != NULL,
+ "%s: subscription() returned NULL list",
+ rd_kafka_name(test_share_consumer_get_rk(rk)));
+
+ return subscription;
+}
+
+
void test_consumer_assign(const char *what,
rd_kafka_t *rk,
rd_kafka_topic_partition_list_t *partitions) {
diff --git a/tests/test.h b/tests/test.h
index 8b5181e9dd..eec532186b 100644
--- a/tests/test.h
+++ b/tests/test.h
@@ -554,6 +554,26 @@ rd_kafka_t *test_create_consumer(
void *opaque),
rd_kafka_conf_t *conf,
rd_kafka_topic_conf_t *default_topic_conf);
+
+rd_kafka_share_t *test_create_share_consumer(const char *group_id);
+
+rd_kafka_t *test_share_consumer_get_rk(rd_kafka_share_t *rkshare);
+
+#define TEST_SHARE_BATCH_SIZE 500
+
+int test_share_consume_batch(rd_kafka_share_t *rk,
+ int timeout_ms,
+ const char **expected_topics,
+ int expected_topic_cnt,
+ int *out_valid);
+
+int test_share_consume_msgs(rd_kafka_share_t *rk,
+ int expected,
+ int max_attempts,
+ int timeout_ms,
+ const char **expected_topics,
+ int expected_topic_cnt);
+
rd_kafka_topic_t *test_create_consumer_topic(rd_kafka_t *rk, const char *topic);
rd_kafka_topic_t *
test_create_topic_object(rd_kafka_t *rk, const char *topic, ...);
@@ -594,6 +614,12 @@ void test_consumer_subscribe(rd_kafka_t *rk, const char *topic);
void test_consumer_subscribe_multi(rd_kafka_t *rk, int topic_count, ...);
+void test_share_consumer_subscribe_multi(rd_kafka_share_t *rk,
+ int topic_count,
+ ...);
+
+rd_kafka_topic_partition_list_t *test_get_subscription(rd_kafka_share_t *rk);
+
void test_consume_msgs_easy_mv0(const char *group_id,
const char *topic,
rd_bool_t txn,
diff --git a/win32/librdkafka.vcxproj b/win32/librdkafka.vcxproj
index b31f895d62..9f30d0652a 100644
--- a/win32/librdkafka.vcxproj
+++ b/win32/librdkafka.vcxproj
@@ -237,6 +237,7 @@
+
diff --git a/win32/tests/tests.vcxproj b/win32/tests/tests.vcxproj
index 812a2674d1..604ed1fc1e 100644
--- a/win32/tests/tests.vcxproj
+++ b/win32/tests/tests.vcxproj
@@ -234,6 +234,8 @@
+
+