@@ -91,8 +91,10 @@ static int rd_kafka_sasl_cyrus_recv(struct rd_kafka_transport_s *rktrans,
91
91
const char * out ;
92
92
unsigned int outlen ;
93
93
94
+ mtx_lock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
94
95
r = sasl_client_step (state -> conn , size > 0 ? buf : NULL , size ,
95
96
& interact , & out , & outlen );
97
+ mtx_unlock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
96
98
97
99
if (r >= 0 ) {
98
100
/* Note: outlen may be 0 here for an empty response */
@@ -148,9 +150,11 @@ static int rd_kafka_sasl_cyrus_recv(struct rd_kafka_transport_s *rktrans,
148
150
RD_KAFKA_DBG_SECURITY ) {
149
151
const char * user , * mech , * authsrc ;
150
152
153
+ mtx_lock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
151
154
if (sasl_getprop (state -> conn , SASL_USERNAME ,
152
155
(const void * * )& user ) != SASL_OK )
153
156
user = "(unknown)" ;
157
+ mtx_unlock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
154
158
155
159
if (sasl_getprop (state -> conn , SASL_MECHNAME ,
156
160
(const void * * )& mech ) != SASL_OK )
@@ -356,6 +360,12 @@ static int rd_kafka_sasl_cyrus_cb_getsimple(void *context,
356
360
switch (id ) {
357
361
case SASL_CB_USER :
358
362
case SASL_CB_AUTHNAME :
363
+ /* Since cyrus expects the returned pointer to be stable
364
+ * and not have its content changed, but the username
365
+ * and password may be updated at anytime by the application
366
+ * calling sasl_set_credentials(), we need to lock
367
+ * rk_conf.sasl.lock before each call into cyrus-sasl.
368
+ * So when we get here the lock is already held. */
359
369
* result = rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .username ;
360
370
break ;
361
371
@@ -381,6 +391,7 @@ static int rd_kafka_sasl_cyrus_cb_getsecret(sasl_conn_t *conn,
381
391
rd_kafka_transport_t * rktrans = context ;
382
392
const char * password ;
383
393
394
+ /* rk_conf.sasl.lock is already locked */
384
395
password = rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .password ;
385
396
386
397
if (!password ) {
@@ -472,8 +483,11 @@ static void rd_kafka_sasl_cyrus_close(struct rd_kafka_transport_s *rktrans) {
472
483
if (!state )
473
484
return ;
474
485
475
- if (state -> conn )
486
+ if (state -> conn ) {
487
+ mtx_lock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
476
488
sasl_dispose (& state -> conn );
489
+ mtx_unlock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
490
+ }
477
491
rd_free (state );
478
492
}
479
493
@@ -528,9 +542,11 @@ static int rd_kafka_sasl_cyrus_client_new(rd_kafka_transport_t *rktrans,
528
542
529
543
memcpy (state -> callbacks , callbacks , sizeof (callbacks ));
530
544
545
+ mtx_lock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
531
546
r = sasl_client_new (rk -> rk_conf .sasl .service_name , hostname , NULL ,
532
547
NULL , /* no local & remote IP checks */
533
548
state -> callbacks , 0 , & state -> conn );
549
+ mtx_unlock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
534
550
if (r != SASL_OK ) {
535
551
rd_snprintf (errstr , errstr_size , "%s" ,
536
552
sasl_errstring (r , NULL , NULL ));
@@ -550,8 +566,10 @@ static int rd_kafka_sasl_cyrus_client_new(rd_kafka_transport_t *rktrans,
550
566
unsigned int outlen ;
551
567
const char * mech = NULL ;
552
568
569
+ mtx_lock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
553
570
r = sasl_client_start (state -> conn , rk -> rk_conf .sasl .mechanisms ,
554
571
NULL , & out , & outlen , & mech );
572
+ mtx_unlock (& rktrans -> rktrans_rkb -> rkb_rk -> rk_conf .sasl .lock );
555
573
556
574
if (r >= 0 )
557
575
if (rd_kafka_sasl_send (rktrans , out , outlen , errstr ,
0 commit comments