@@ -2666,8 +2666,8 @@ static void do_test_topic_disappears_for_awhile(void) {
26662666 SUB_TEST_QUICK ();
26672667
26682668 rk = create_txn_producer (
2669- & mcluster , txnid , 1 , NULL , "batch.num.messages" , "3" , "linger.ms" ,
2670- "100" , " topic.metadata.refresh.interval.ms" , "2000" , NULL );
2669+ & mcluster , txnid , 1 , "batch.num.messages" , "3" , "linger.ms" , "100 " ,
2670+ "topic.metadata.refresh.interval.ms" , "2000" , NULL );
26712671
26722672 rd_kafka_mock_topic_create (mcluster , topic , partition_cnt , 1 );
26732673
@@ -2742,6 +2742,104 @@ static void do_test_topic_disappears_for_awhile(void) {
27422742}
27432743
27442744
2745+ /**
2746+ * @brief Test that group coordinator requests can handle an
2747+ * untimely disconnect.
2748+ *
2749+ * The transaction manager makes use of librdkafka coord_req to commit
2750+ * transaction offsets to the group coordinator.
2751+ * If the connection to the given group coordinator is not up the
2752+ * coord_req code will request a connection once, but if this connection fails
2753+ * there will be no new attempts and the coord_req will idle until either
2754+ * destroyed or the connection is retried for other reasons.
2755+ * This in turn stalls the send_offsets_to_transaction() call until the
2756+ * transaction times out.
2757+ */
2758+ static int delayed_up_cb (void * arg ) {
2759+ rd_kafka_mock_cluster_t * mcluster = arg ;
2760+ rd_sleep (3 );
2761+ TEST_SAY ("Bringing up group coordinator 2..\n" );
2762+ rd_kafka_mock_broker_set_up (mcluster , 2 );
2763+ return 0 ;
2764+ }
2765+
2766+ static void do_test_disconnected_group_coord (void ) {
2767+ const char * topic = "mytopic" ;
2768+ const char * txnid = "myTxnId" ;
2769+ const char * grpid = "myGrpId" ;
2770+ const int partition_cnt = 1 ;
2771+ rd_kafka_t * rk ;
2772+ rd_kafka_mock_cluster_t * mcluster ;
2773+ rd_kafka_topic_partition_list_t * offsets ;
2774+ rd_kafka_consumer_group_metadata_t * cgmetadata ;
2775+ test_timing_t timing ;
2776+ thrd_t thrd ;
2777+ int ret ;
2778+
2779+ SUB_TEST_QUICK ();
2780+
2781+ test_curr -> is_fatal_cb = error_is_fatal_cb ;
2782+ allowed_error = RD_KAFKA_RESP_ERR__TRANSPORT ;
2783+
2784+ rk = create_txn_producer (& mcluster , txnid , 3 , NULL );
2785+
2786+ rd_kafka_mock_topic_create (mcluster , topic , partition_cnt , 1 );
2787+
2788+ /* Broker 1: txn coordinator
2789+ * Broker 2: group coordinator
2790+ * Broker 3: partition leader */
2791+ rd_kafka_mock_coordinator_set (mcluster , "transaction" , txnid , 1 );
2792+ rd_kafka_mock_coordinator_set (mcluster , "group" , grpid , 2 );
2793+ rd_kafka_mock_partition_set_leader (mcluster , topic , 0 , 3 );
2794+
2795+ /* Bring down group coordinator so there are no undesired
2796+ * connections to it. */
2797+ rd_kafka_mock_broker_set_down (mcluster , 2 );
2798+
2799+
2800+ TEST_CALL_ERROR__ (rd_kafka_init_transactions (rk , -1 ));
2801+ TEST_CALL_ERROR__ (rd_kafka_begin_transaction (rk ));
2802+ TEST_CALL_ERR__ (rd_kafka_producev (
2803+ rk , RD_KAFKA_V_TOPIC (topic ), RD_KAFKA_V_PARTITION (0 ),
2804+ RD_KAFKA_V_VALUE ("hi" , 2 ), RD_KAFKA_V_END ));
2805+ test_flush (rk , -1 );
2806+
2807+ rd_sleep (1 );
2808+
2809+ /* Run a background thread that after 3s, which should be enough
2810+ * to perform the first failed connection attempt, makes the
2811+ * group coordinator available again. */
2812+ thrd_create (& thrd , delayed_up_cb , mcluster );
2813+
2814+ TEST_SAY ("Calling send_offsets_to_transaction()\n" );
2815+ offsets = rd_kafka_topic_partition_list_new (1 );
2816+ rd_kafka_topic_partition_list_add (offsets , topic , 0 )-> offset = 1 ;
2817+ cgmetadata = rd_kafka_consumer_group_metadata_new (grpid );
2818+
2819+ TIMING_START (& timing , "send_offsets_to_transaction(-1)" );
2820+ TEST_CALL_ERROR__ (
2821+ rd_kafka_send_offsets_to_transaction (rk , offsets , cgmetadata , -1 ));
2822+ TIMING_STOP (& timing );
2823+ TIMING_ASSERT (& timing , 0 , 10 * 1000 /*10s*/ );
2824+
2825+ rd_kafka_consumer_group_metadata_destroy (cgmetadata );
2826+ rd_kafka_topic_partition_list_destroy (offsets );
2827+ thrd_join (thrd , & ret );
2828+
2829+ /* Commit the transaction */
2830+ TIMING_START (& timing , "commit_transaction(-1)" );
2831+ TEST_CALL_ERROR__ (rd_kafka_commit_transaction (rk , -1 ));
2832+ TIMING_STOP (& timing );
2833+
2834+ rd_kafka_destroy (rk );
2835+
2836+ allowed_error = RD_KAFKA_RESP_ERR_NO_ERROR ;
2837+ test_curr -> is_fatal_cb = NULL ;
2838+
2839+ SUB_TEST_PASS ();
2840+ }
2841+
2842+
27452843int main_0105_transactions_mock (int argc , char * * argv ) {
27462844 if (test_needs_auth ()) {
27472845 TEST_SKIP ("Mock cluster does not support SSL/SASL\n" );
@@ -2815,5 +2913,7 @@ int main_0105_transactions_mock(int argc, char **argv) {
28152913
28162914 do_test_topic_disappears_for_awhile ();
28172915
2916+ do_test_disconnected_group_coord ();
2917+
28182918 return 0 ;
28192919}
0 commit comments