@@ -94,6 +94,95 @@ do_test_produce_consumer_with_OIDC(const char *test_name,
9494}
9595
9696
97+ /**
98+ * @brief After config OIDC, make sure the share consumer
99+ * can work successfully.
100+ */
101+ static void
102+ do_test_produce_share_consumer_with_OIDC (const char * test_name ,
103+ const rd_kafka_conf_t * base_conf ) {
104+ const char * topic ;
105+ rd_kafka_t * p1 ;
106+ rd_kafka_share_t * sc1 ;
107+ rd_kafka_conf_t * conf ;
108+ rd_kafka_topic_partition_list_t * subs ;
109+ rd_kafka_message_t * batch [500 ];
110+ const char * grp_conf [] = {"share.auto.offset.reset" , "SET" , "earliest" };
111+ int consumed = 0 , attempts ;
112+ const int msg_cnt = 10 ;
113+
114+ const char * url = test_getenv ("VALID_OIDC_URL" , NULL );
115+
116+ SUB_TEST (
117+ "Test share consumer with oidc configuration: %s" , test_name );
118+
119+ if (!url ) {
120+ SUB_TEST_SKIP (
121+ "VALID_OIDC_URL environment variable is not set\n" );
122+ return ;
123+ }
124+
125+ conf = rd_kafka_conf_dup (base_conf );
126+ test_conf_set (conf , "sasl.oauthbearer.token.endpoint.url" , url );
127+
128+ rd_kafka_conf_set_dr_msg_cb (conf , test_dr_msg_cb );
129+
130+ p1 = test_create_handle (RD_KAFKA_PRODUCER , conf );
131+
132+ topic = test_mk_topic_name ("0126-oauthbearer_oidc_share" , 1 );
133+ test_create_topic_wait_exists (p1 , topic , 1 , 3 , 5000 );
134+ TEST_SAY ("Topic: %s is created\n" , topic );
135+
136+ test_produce_msgs_easy (topic , 0 , 0 , msg_cnt );
137+
138+ /* Create share consumer (picks up SASL config from test_conf_init) */
139+ sc1 = test_create_share_consumer ("oidc-share-group" );
140+
141+ /* Set group config for earliest offset */
142+ test_IncrementalAlterConfigs_simple (
143+ p1 , RD_KAFKA_RESOURCE_GROUP , "oidc-share-group" , grp_conf , 1 );
144+
145+ /* Subscribe */
146+ subs = rd_kafka_topic_partition_list_new (1 );
147+ rd_kafka_topic_partition_list_add (subs , topic , RD_KAFKA_PARTITION_UA );
148+ rd_kafka_share_subscribe (sc1 , subs );
149+ rd_kafka_topic_partition_list_destroy (subs );
150+
151+ /* Give it some time to trigger the token refresh. */
152+ rd_usleep (5 * 1000 * 1000 , NULL );
153+
154+ /* Consume messages */
155+ attempts = 50 ;
156+ while (consumed < msg_cnt && attempts -- > 0 ) {
157+ size_t rcvd = 0 ;
158+ size_t m ;
159+ rd_kafka_error_t * err ;
160+
161+ err = rd_kafka_share_consume_batch (sc1 , 3000 , batch , & rcvd );
162+ if (err ) {
163+ rd_kafka_error_destroy (err );
164+ continue ;
165+ }
166+
167+ for (m = 0 ; m < rcvd ; m ++ ) {
168+ if (!batch [m ]-> err )
169+ consumed ++ ;
170+ rd_kafka_message_destroy (batch [m ]);
171+ }
172+ }
173+
174+ TEST_ASSERT (consumed == msg_cnt ,
175+ "Expected %d messages, consumed %d" , msg_cnt , consumed );
176+ TEST_SAY ("Share consumer consumed %d/%d messages with OIDC\n" ,
177+ consumed , msg_cnt );
178+
179+ rd_kafka_share_consumer_close (sc1 );
180+ rd_kafka_share_destroy (sc1 );
181+ rd_kafka_destroy (p1 );
182+ SUB_TEST_PASS ();
183+ }
184+
185+
97186static void
98187auth_error_cb (rd_kafka_t * rk , int err , const char * reason , void * opaque ) {
99188 if (err == RD_KAFKA_RESP_ERR__AUTHENTICATION ||
@@ -149,6 +238,59 @@ static void do_test_produce_consumer_with_OIDC_expired_token_should_fail(
149238}
150239
151240
241+ /**
242+ * @brief After config OIDC with expired token, make sure the share consumer
243+ * authentication fails as expected.
244+ */
245+ static void
246+ do_test_produce_share_consumer_with_OIDC_expired_token_should_fail (
247+ const rd_kafka_conf_t * base_conf ) {
248+ rd_kafka_share_t * sc1 ;
249+ rd_kafka_conf_t * conf ;
250+ rd_kafka_message_t * batch [10 ];
251+ size_t rcvd = 0 ;
252+ rd_kafka_error_t * err ;
253+ char errstr [512 ];
254+
255+ const char * expired_url = test_getenv ("EXPIRED_TOKEN_OIDC_URL" , NULL );
256+
257+ SUB_TEST ("Test OAUTHBEARER/OIDC share consumer failing with "
258+ "expired JWT" );
259+
260+ if (!expired_url ) {
261+ SUB_TEST_SKIP (
262+ "EXPIRED_TOKEN_OIDC_URL environment variable is not "
263+ "set\n" );
264+ return ;
265+ }
266+
267+ conf = rd_kafka_conf_dup (base_conf );
268+
269+ error_seen = rd_false ;
270+ test_conf_set (conf , "sasl.oauthbearer.token.endpoint.url" ,
271+ expired_url );
272+ test_conf_set (conf , "group.id" , "oidc-share-fail" );
273+
274+ rd_kafka_conf_set_error_cb (conf , auth_error_cb );
275+
276+ sc1 = rd_kafka_share_consumer_new (conf , errstr , sizeof (errstr ));
277+ TEST_ASSERT (sc1 , "Failed to create share consumer: %s" , errstr );
278+
279+ /* Poll — should trigger auth error callback, no messages expected */
280+ err = rd_kafka_share_consume_batch (sc1 , 10 * 1000 , batch , & rcvd );
281+ if (err )
282+ rd_kafka_error_destroy (err );
283+
284+ TEST_ASSERT (error_seen ,
285+ "Expected authentication error for share consumer "
286+ "with expired token" );
287+
288+ rd_kafka_share_consumer_close (sc1 );
289+ rd_kafka_share_destroy (sc1 );
290+ SUB_TEST_PASS ();
291+ }
292+
293+
152294/**
153295 * @brief After configiguring OIDC, make sure the
154296 * authentication fails as expected.
@@ -233,6 +375,58 @@ do_test_produce_consumer_with_OIDC_should_fail_invalid_token_endpoint(
233375 SUB_TEST_PASS ();
234376}
235377
378+
379+ /**
380+ * @brief After config OIDC with invalid token endpoint, make sure the
381+ * share consumer authentication fails as expected.
382+ */
383+ static void
384+ do_test_produce_share_consumer_with_OIDC_should_fail_invalid_token_endpoint (
385+ const rd_kafka_conf_t * base_conf ) {
386+ rd_kafka_share_t * sc1 ;
387+ rd_kafka_conf_t * conf ;
388+ rd_kafka_message_t * batch [10 ];
389+ size_t rcvd = 0 ;
390+ rd_kafka_error_t * err ;
391+ char errstr [512 ];
392+
393+ const char * invalid_url = test_getenv ("INVALID_OIDC_URL" , NULL );
394+
395+ SUB_TEST ("Test OAUTHBEARER/OIDC share consumer failing with "
396+ "invalid JWT" );
397+
398+ if (!invalid_url ) {
399+ SUB_TEST_SKIP (
400+ "INVALID_OIDC_URL environment variable is not set\n" );
401+ return ;
402+ }
403+
404+ conf = rd_kafka_conf_dup (base_conf );
405+
406+ error_seen = rd_false ;
407+ test_conf_set (conf , "sasl.oauthbearer.token.endpoint.url" ,
408+ invalid_url );
409+ test_conf_set (conf , "group.id" , "oidc-share-fail-invalid" );
410+
411+ rd_kafka_conf_set_error_cb (conf , auth_error_cb );
412+
413+ sc1 = rd_kafka_share_consumer_new (conf , errstr , sizeof (errstr ));
414+ TEST_ASSERT (sc1 , "Failed to create share consumer: %s" , errstr );
415+
416+ /* Poll — should trigger auth error callback, no messages expected */
417+ err = rd_kafka_share_consume_batch (sc1 , 10 * 1000 , batch , & rcvd );
418+ if (err )
419+ rd_kafka_error_destroy (err );
420+
421+ TEST_ASSERT (error_seen ,
422+ "Expected authentication error for share consumer "
423+ "with invalid token endpoint" );
424+
425+ rd_kafka_share_consumer_close (sc1 );
426+ rd_kafka_share_destroy (sc1 );
427+ SUB_TEST_PASS ();
428+ }
429+
236430typedef enum oidc_configuration_jwt_bearer_variation_t {
237431 /** Use a private key file. */
238432 OIDC_CONFIGURATION_JWT_BEARER_VARIATION_PRIVATE_KEY_FILE ,
@@ -516,9 +710,13 @@ int main_0126_oauthbearer_oidc(int argc, char **argv) {
516710 }
517711
518712 do_test_produce_consumer_with_OIDC ("client_credentials" , conf );
713+ do_test_produce_share_consumer_with_OIDC ("client_credentials" , conf );
519714 do_test_produce_consumer_with_OIDC_should_fail_invalid_token_endpoint (
520715 conf );
716+ do_test_produce_share_consumer_with_OIDC_should_fail_invalid_token_endpoint (
717+ conf );
521718 do_test_produce_consumer_with_OIDC_expired_token_should_fail (conf );
719+ do_test_produce_share_consumer_with_OIDC_expired_token_should_fail (conf );
522720 do_test_produce_consumer_with_OIDC_jwt_bearer (conf );
523721 do_test_produce_consumer_with_OIDC_metadata_authentication (conf );
524722
0 commit comments