1+ /*
2+ * librdkafka - Apache Kafka C library
3+ *
4+ * Copyright (c) 2025, Confluent Inc.
5+ * All rights reserved.
6+ *
7+ * Redistribution and use in source and binary forms, with or without
8+ * modification, are permitted provided that the following conditions are met:
9+ *
10+ * 1. Redistributions of source code must retain the above copyright notice,
11+ * this list of conditions and the following disclaimer.
12+ * 2. Redistributions in binary form must reproduce the above copyright notice,
13+ * this list of conditions and the following disclaimer in the documentation
14+ * and/or other materials provided with the distribution.
15+ *
16+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
17+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
18+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
19+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
20+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
21+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
22+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
23+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
24+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
25+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
26+ * POSSIBILITY OF SUCH DAMAGE.
27+ */
28+
29+ #include <stdio.h>
30+ #include <string.h>
31+ #include "test.h"
32+ #include "rdkafka.h"
33+
34+ /**
35+ * Integration test for KIP-848 partition metadata refresh:
36+ * - Create topic with 2 partitions
37+ * - Start consumer group and verify initial assignment
38+ * - Increase partition count to 4
39+ * - Reset log tracking variables after partition creation
40+ * - Wait for HeartbeatRequest, HeartbeatResponse, and metadata refresh logs
41+ * - Assert that metadata refresh is triggered for new partitions
42+ */
43+
44+ // Globals to track log sequence
45+ static volatile int seen_heartbeat_req = 0 ;
46+ static volatile int seen_heartbeat_resp = 0 ;
47+ static volatile int seen_metadata_log = 0 ;
48+
49+ static void reset_log_tracking (void ) {
50+ seen_heartbeat_req = 0 ;
51+ seen_heartbeat_resp = 0 ;
52+ seen_metadata_log = 0 ;
53+ }
54+
55+ static void wait_for_metadata_refresh_log (int timeout_ms ) {
56+ int elapsed = 0 ;
57+ while (elapsed < timeout_ms && !seen_metadata_log ) {
58+ rd_usleep (500 * 1000 , NULL ); // 500 ms
59+ elapsed += 500 ;
60+ }
61+ TEST_ASSERT (seen_heartbeat_req , "Expected HeartbeatRequest log not seen after partition creation" );
62+ TEST_ASSERT (seen_heartbeat_resp , "Expected HeartbeatResponse log not seen after partition creation" );
63+ TEST_ASSERT (seen_metadata_log , "Expected metadata refresh log not seen after partition creation and heartbeat" );
64+ }
65+
66+ // Custom log callback to capture and process librdkafka logs
67+ static void test_metadata_log_cb (const rd_kafka_t * rk , int level , const char * fac , const char * buf ) {
68+ if (strstr (buf , "Sent ConsumerGroupHeartbeatRequest" )) {
69+ seen_heartbeat_req = 1 ;
70+ }
71+ if (seen_heartbeat_req && strstr (buf , "Received ConsumerGroupHeartbeatResponse" )) {
72+ seen_heartbeat_resp = 1 ;
73+ }
74+ if (seen_heartbeat_resp && strstr (buf , "Partition assigned to this consumer is not present in cached metadata" )) {
75+ seen_metadata_log = 1 ;
76+ }
77+ }
78+
79+ static rd_kafka_t * create_consumer (const char * topic , const char * group , void (* log_cb )(const rd_kafka_t * , int , const char * , const char * )) {
80+ rd_kafka_conf_t * conf ;
81+ test_conf_init (& conf , NULL , 60 );
82+ test_conf_set (conf , "group.id" , group );
83+ test_conf_set (conf , "auto.offset.reset" , "earliest" );
84+ test_conf_set (conf , "debug" , "cgrp, protocol" );
85+ rd_kafka_conf_set_log_cb (conf , test_metadata_log_cb );
86+ rd_kafka_t * consumer = test_create_consumer (topic , NULL , conf , NULL );
87+ return consumer ;
88+ }
89+
90+ static void setup_and_run_metadata_refresh_test (void ) {
91+ const char * topic = test_mk_topic_name ("cgrp_metadata" , 1 );
92+ int initial_partitions = 2 ;
93+ int new_partitions = 4 ;
94+ rd_kafka_t * c1 , * c2 , * rk ;
95+ const char * group = "grp_metadata" ;
96+
97+ SUB_TEST_QUICK ();
98+
99+ TEST_SAY ("Creating topic %s with %d partitions\n" , topic , initial_partitions );
100+ test_create_topic (NULL , topic , initial_partitions , 1 );
101+
102+ TEST_SAY ("Creating consumers\n" );
103+ c1 = create_consumer (topic , group , test_metadata_log_cb );
104+ c2 = create_consumer (topic , group , test_metadata_log_cb );
105+
106+ rk = test_create_handle (RD_KAFKA_PRODUCER , NULL );
107+
108+ TEST_SAY ("Subscribing to topic %s\n" , topic );
109+ test_consumer_subscribe (c1 , topic );
110+ test_consumer_subscribe (c2 , topic );
111+
112+ // Wait for initial assignment
113+ test_consumer_wait_assignment (c1 , rd_false );
114+ test_consumer_wait_assignment (c2 , rd_false );
115+
116+ // Create new partitions
117+ TEST_SAY ("Increasing partition count to %d\n" , new_partitions );
118+ test_create_partitions (rk , topic , new_partitions );
119+
120+ // Reset log tracking variables to only consider logs after partition creation
121+ reset_log_tracking ();
122+
123+ // Wait for expected logs for up to 10 seconds
124+ wait_for_metadata_refresh_log (10000 );
125+
126+ TEST_SAY ("Closing consumers\n" );
127+ test_consumer_close (c1 );
128+ test_consumer_close (c2 );
129+ rd_kafka_destroy (c1 );
130+ rd_kafka_destroy (c2 );
131+
132+ SUB_TEST_PASS ();
133+ }
134+
135+ int main_0154_metadata_refresh (int argc , char * * argv ) {
136+ if (!test_consumer_group_protocol_classic ())
137+ setup_and_run_metadata_refresh_test ();
138+ return 0 ;
139+ }
0 commit comments